From 81d324da152b8411396444b760e6332ec8a2315a Mon Sep 17 00:00:00 2001 From: Stig Bakken Date: Fri, 12 Feb 2021 15:12:14 +0800 Subject: [PATCH 01/34] MaterializeMySQL: add skipping index for _version column --- .../MySQL/InterpretersMySQLDDLQuery.cpp | 16 +++- .../MySQL/tests/gtest_create_rewritten.cpp | 82 +++++++++++-------- 2 files changed, 64 insertions(+), 34 deletions(-) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 7f4da0638d4..dfc126a6c24 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -411,13 +412,26 @@ ASTs InterpreterCreateImpl::getRewrittenQueries( return column_declaration; }; - /// Add _sign and _version column. + /// Add _sign and _version columns. String sign_column_name = getUniqueColumnName(columns_name_and_type, "_sign"); String version_column_name = getUniqueColumnName(columns_name_and_type, "_version"); columns->set(columns->columns, InterpreterCreateQuery::formatColumns(columns_name_and_type)); columns->columns->children.emplace_back(create_materialized_column_declaration(sign_column_name, "Int8", UInt64(1))); columns->columns->children.emplace_back(create_materialized_column_declaration(version_column_name, "UInt64", UInt64(1))); + /// Add minmax skipping index for _version column. + auto version_index = std::make_shared(); + version_index->name = version_column_name; + auto index_expr = std::make_shared(version_column_name); + auto index_type = makeASTFunction("minmax"); + index_type->no_empty_args = true; + version_index->set(version_index->expr, index_expr); + version_index->set(version_index->type, index_type); + version_index->granularity = 1; + ASTPtr indices = std::make_shared(); + indices->children.push_back(version_index); + columns->set(columns->indices, indices); + auto storage = std::make_shared(); /// The `partition by` expression must use primary keys, otherwise the primary keys will not be merge. diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index 0d8e57aafc5..5a82a570db0 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -28,6 +28,10 @@ static inline ASTPtr tryRewrittenCreateQuery(const String & query, const Context context, "test_database", "test_database")[0]; } +static const char MATERIALIZEMYSQL_TABLE_COLUMNS[] = ", `_sign` Int8() MATERIALIZED 1" + ", `_version` UInt64() MATERIALIZED 1" + ", INDEX _version _version TYPE minmax GRANULARITY 1"; + TEST(MySQLCreateRewritten, ColumnsDataType) { tryRegisterFunctions(); @@ -45,46 +49,46 @@ TEST(MySQLCreateRewritten, ColumnsDataType) { EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + ")", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` Nullable(" + mapped_type + ")" - ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` Nullable(" + mapped_type + ")" + + MATERIALIZEMYSQL_TABLE_COLUMNS + ") ENGINE = " "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " NOT NULL)", context_holder.context)), "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` " + mapped_type + - ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + MATERIALIZEMYSQL_TABLE_COLUMNS + ") ENGINE = " "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " COMMENT 'test_comment' NOT NULL)", context_holder.context)), "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` " + mapped_type + - ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + MATERIALIZEMYSQL_TABLE_COLUMNS + ") ENGINE = " "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); if (Poco::toUpper(test_type).find("INT") != std::string::npos) { EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " UNSIGNED)", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` Nullable(U" + mapped_type + ")" - ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` Nullable(U" + mapped_type + ")" + + MATERIALIZEMYSQL_TABLE_COLUMNS + ") ENGINE = " "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " COMMENT 'test_comment' UNSIGNED)", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` Nullable(U" + mapped_type + ")" - ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` Nullable(U" + mapped_type + ")" + + MATERIALIZEMYSQL_TABLE_COLUMNS + ") ENGINE = " "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " NOT NULL UNSIGNED)", context_holder.context)), "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` U" + mapped_type + - ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + MATERIALIZEMYSQL_TABLE_COLUMNS + ") ENGINE = " "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " COMMENT 'test_comment' UNSIGNED NOT NULL)", context_holder.context)), "CREATE TABLE test_database.test_table_1 (`key` Int32, `test` U" + mapped_type + - ", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + MATERIALIZEMYSQL_TABLE_COLUMNS + ") ENGINE = " "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); } } @@ -109,13 +113,15 @@ TEST(MySQLCreateRewritten, PartitionPolicy) { EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " PRIMARY KEY)", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, " - "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY tuple(key)"); + "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + + MATERIALIZEMYSQL_TABLE_COLUMNS + + ") ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " NOT NULL PRIMARY KEY)", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, " - "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY tuple(key)"); + "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + + MATERIALIZEMYSQL_TABLE_COLUMNS + + ") ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY tuple(key)"); } } @@ -138,23 +144,27 @@ TEST(MySQLCreateRewritten, OrderbyPolicy) { EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " PRIMARY KEY, `key2` " + test_type + " UNIQUE KEY)", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `key2` Nullable(" + mapped_type + "), `_sign` Int8() MATERIALIZED 1, " - "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY (key, assumeNotNull(key2))"); + "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `key2` Nullable(" + mapped_type + ")" + + MATERIALIZEMYSQL_TABLE_COLUMNS + + ") ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY (key, assumeNotNull(key2))"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " NOT NULL PRIMARY KEY, `key2` " + test_type + " NOT NULL UNIQUE KEY)", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `key2` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, " - "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY (key, key2)"); + "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `key2` " + mapped_type + + MATERIALIZEMYSQL_TABLE_COLUMNS + + ") ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY (key, key2)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " KEY UNIQUE KEY)", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, " - "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY tuple(key)"); + "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + + MATERIALIZEMYSQL_TABLE_COLUMNS + + ") ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + ", `key2` " + test_type + " UNIQUE KEY, PRIMARY KEY(`key`, `key2`))", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `key2` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, " - "`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY (key, key2)"); + "CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `key2` " + mapped_type + + MATERIALIZEMYSQL_TABLE_COLUMNS + + ") ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY (key, key2)"); } } @@ -165,23 +175,27 @@ TEST(MySQLCreateRewritten, RewrittenQueryWithPrimaryKey) EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL PRIMARY KEY) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version) " - "PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); + "CREATE TABLE test_database.test_table_1 (`key` Int32" + + std::string(MATERIALIZEMYSQL_TABLE_COLUMNS) + + ") ENGINE = ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL, PRIMARY KEY (`key`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version) " - "PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); + "CREATE TABLE test_database.test_table_1 (`key` Int32" + + std::string(MATERIALIZEMYSQL_TABLE_COLUMNS) + + ") ENGINE = ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1` (`key_1` int NOT NULL, key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key_1` Int32, `key_2` Int32, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " - "ReplacingMergeTree(_version) PARTITION BY intDiv(key_1, 4294967) ORDER BY (key_1, key_2)"); + "CREATE TABLE test_database.test_table_1 (`key_1` Int32, `key_2` Int32" + + std::string(MATERIALIZEMYSQL_TABLE_COLUMNS) + + ") ENGINE = ReplacingMergeTree(_version) PARTITION BY intDiv(key_1, 4294967) ORDER BY (key_1, key_2)"); EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1` (`key_1` BIGINT NOT NULL, key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key_1` Int64, `key_2` Int32, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " - "ReplacingMergeTree(_version) PARTITION BY intDiv(key_2, 4294967) ORDER BY (key_1, key_2)"); + "CREATE TABLE test_database.test_table_1 (`key_1` Int64, `key_2` Int32" + + std::string(MATERIALIZEMYSQL_TABLE_COLUMNS) + + ") ENGINE = ReplacingMergeTree(_version) PARTITION BY intDiv(key_2, 4294967) ORDER BY (key_1, key_2)"); } TEST(MySQLCreateRewritten, RewrittenQueryWithPrefixKey) @@ -191,7 +205,8 @@ TEST(MySQLCreateRewritten, RewrittenQueryWithPrefixKey) EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL PRIMARY KEY, `prefix_key` varchar(200) NOT NULL, KEY prefix_key_index(prefix_key(2))) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`key` Int32, `prefix_key` String, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = " + "CREATE TABLE test_database.test_table_1 (`key` Int32, `prefix_key` String" + + std::string(MATERIALIZEMYSQL_TABLE_COLUMNS) + ") ENGINE = " "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY (key, prefix_key)"); } @@ -204,6 +219,7 @@ TEST(MySQLCreateRewritten, UniqueKeysConvert) "CREATE TABLE `test_database`.`test_table_1` (code varchar(255) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL,name varchar(255) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL," " id bigint NOT NULL AUTO_INCREMENT, tenant_id bigint NOT NULL, PRIMARY KEY (id), UNIQUE KEY code_id (code, tenant_id), UNIQUE KEY name_id (name, tenant_id))" " ENGINE=InnoDB AUTO_INCREMENT=100 DEFAULT CHARSET=utf8 COLLATE=utf8_bin;", context_holder.context)), - "CREATE TABLE test_database.test_table_1 (`code` String, `name` String, `id` Int64, `tenant_id` Int64, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1)" - " ENGINE = ReplacingMergeTree(_version) PARTITION BY intDiv(id, 18446744073709551) ORDER BY (code, name, tenant_id, id)"); + "CREATE TABLE test_database.test_table_1 (`code` String, `name` String, `id` Int64, `tenant_id` Int64" + + std::string(MATERIALIZEMYSQL_TABLE_COLUMNS) + + ") ENGINE = ReplacingMergeTree(_version) PARTITION BY intDiv(id, 18446744073709551) ORDER BY (code, name, tenant_id, id)"); } From a4d3732091b2b8d85c997cd030032978a845f4b4 Mon Sep 17 00:00:00 2001 From: turbo jason Date: Fri, 19 Mar 2021 20:51:29 +0800 Subject: [PATCH 02/34] [ClickHouse][Copier] Improve copier work --- programs/copier/ClusterCopier.cpp | 170 ++++++++++++++++++++---------- programs/copier/ClusterCopier.h | 9 +- programs/copier/TaskCluster.h | 1 + 3 files changed, 124 insertions(+), 56 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 7eea23160b2..5014f75ae48 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -600,10 +600,14 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t Settings settings_push = task_cluster->settings_push; - /// It is important, ALTER ATTACH PARTITION must be done synchronously - /// And we will execute this ALTER query on each replica of a shard. - /// It is correct, because this query is idempotent. - settings_push.replication_alter_partitions_sync = 2; + Settings settings_push = task_cluster->settings_push; + ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_NODE; + UInt64 max_successful_executions_per_shard = 0; + if (settings_push.replication_alter_partitions_sync == 1) + { + execution_mode = ClusterExecutionMode::ON_EACH_SHARD; + max_successful_executions_per_shard = 1; + } query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + ((partition_name == "'all'") ? " ATTACH PARTITION ID " : " ATTACH PARTITION ") + partition_name + @@ -613,14 +617,33 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t try { - size_t num_nodes = executeQueryOnCluster( - task_table.cluster_push, - query_alter_ast_string, - settings_push, - PoolMode::GET_MANY, - ClusterExecutionMode::ON_EACH_NODE); + /// Try attach partition on each shard + UInt64 num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_alter_ast_string, + task_cluster->settings_push, + PoolMode::GET_MANY, + execution_mode, + max_successful_executions_per_shard); - LOG_INFO(log, "Number of nodes that executed ALTER query successfully : {}", toString(num_nodes)); + if (settings_push.replication_alter_partitions_sync == 1) + { + LOG_INFO( + log, + "Destination tables {} have been executed alter query successfuly on {} shards of {}", + getQuotedTable(task_table.table_push), + num_nodes, + task_table.cluster_push->getShardCount()); + + if (num_nodes != task_table.cluster_push->getShardCount()) + { + return TaskStatus::Error; + } + } + else + { + LOG_INFO(log, "Number of nodes that executed ALTER query successfully : {}", toString(num_nodes)); + } } catch (...) { @@ -856,6 +879,16 @@ bool ClusterCopier::tryDropPartitionPiece( bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table) { + //first create destionation table + TaskStatus task_status = TaskStatus::Error; + + task_status = tryCreateDestinationTable(timeouts, task_table); + /// Exit if success + if (task_status != TaskStatus::Finished) + { + LOG_WARNING(log, "Create destination Tale Failed "); + return false; + } /// An heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint bool previous_shard_is_instantly_finished = false; @@ -932,7 +965,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Do not sleep if there is a sequence of already processed shards to increase startup bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote; - TaskStatus task_status = TaskStatus::Error; + task_status = TaskStatus::Error; bool was_error = false; has_shard_to_process = true; for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) @@ -1050,6 +1083,44 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab return table_is_done; } +TaskStatus ClusterCopier::tryCreateDestinationTable(const ConnectionTimeouts & timeouts, TaskTable & task_table) +{ + /// Try create original table (if not exists) on each shard + + //TaskTable & task_table = task_shard.task_table; + const TaskShardPtr task_shard = task_table.all_shards.at(0); + /// We need to update table definitions for each part, it could be changed after ALTER + task_shard->current_pull_table_create_query = getCreateTableForPullShard(timeouts, *task_shard); + try + { + auto create_query_push_ast + = rewriteCreateQueryStorage(task_shard->current_pull_table_create_query, task_table.table_push, task_table.engine_push_ast); + auto & create = create_query_push_ast->as(); + create.if_not_exists = true; + InterpreterCreateQuery::prepareOnClusterQuery(create, context, task_table.cluster_push_name); + String query = queryToString(create_query_push_ast); + + LOG_DEBUG(log, "Create destination tables. Query: {}", query); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, PoolMode::GET_MANY); + LOG_INFO( + log, + "Destination tables {} have been created on {} shards of {}", + getQuotedTable(task_table.table_push), + shards, + task_table.cluster_push->getShardCount()); + if (shards != task_table.cluster_push->getShardCount()) + { + return TaskStatus::Error; + } + } + catch (...) + { + tryLogCurrentException(log, "Error while creating original table. Maybe we are not first."); + } + + return TaskStatus::Finished; +} + /// Job for copying partition from particular shard. TaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) { @@ -1366,8 +1437,17 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( LOG_DEBUG(log, "Create destination tables. Query: {}", query); UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, PoolMode::GET_MANY); - LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", - getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount()); + LOG_INFO( + log, + "Destination tables {} have been created on {} shards of {}", + getQuotedTable(task_table.table_push), + shards, + task_table.cluster_push->getShardCount()); + + if (shards != task_table.cluster_push->getShardCount()) + { + return TaskStatus::Error; + } } /// Do the copying @@ -1471,32 +1551,13 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( { tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); create_is_dirty_node(clean_state_clock); + dropHelpingTablesByPieceNumber(task_table, current_piece_number); return TaskStatus::Error; } } LOG_INFO(log, "Partition {} piece {} copied. But not moved to original destination table.", task_partition.name, toString(current_piece_number)); - - /// Try create original table (if not exists) on each shard - try - { - auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, - task_table.table_push, task_table.engine_push_ast); - auto & create = create_query_push_ast->as(); - create.if_not_exists = true; - InterpreterCreateQuery::prepareOnClusterQuery(create, context, task_table.cluster_push_name); - String query = queryToString(create_query_push_ast); - - LOG_DEBUG(log, "Create destination tables. Query: {}", query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, PoolMode::GET_MANY); - LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount()); - } - catch (...) - { - tryLogCurrentException(log, "Error while creating original table. Maybe we are not first."); - } - /// Finalize the processing, change state of current partition task (and also check is_dirty flag) { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); @@ -1538,33 +1599,36 @@ void ClusterCopier::dropLocalTableIfExists(const DatabaseAndTableName & table_na interpreter.execute(); } +void ClusterCopier::dropHelpingTablesByPieceNumber(const TaskTable & task_table, size_t current_piece_number) +{ + LOG_DEBUG(log, "Removing helping tables piece {}", current_piece_number); + + DatabaseAndTableName original_table = task_table.table_push; + DatabaseAndTableName helping_table + = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); + + String query = "DROP TABLE IF EXISTS " + getQuotedTable(helping_table); + + const ClusterPtr & cluster_push = task_table.cluster_push; + Settings settings_push = task_cluster->settings_push; + + LOG_DEBUG(log, "Execute distributed DROP TABLE: {}", query); + + /// We have to drop partition_piece on each replica + UInt64 num_nodes = executeQueryOnCluster(cluster_push, query, settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); + + LOG_INFO(log, "DROP TABLE query was successfully executed on {} nodes.", toString(num_nodes)); +} void ClusterCopier::dropHelpingTables(const TaskTable & task_table) { LOG_DEBUG(log, "Removing helping tables"); for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) { - DatabaseAndTableName original_table = task_table.table_push; - DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); - - String query = "DROP TABLE IF EXISTS " + getQuotedTable(helping_table); - - const ClusterPtr & cluster_push = task_table.cluster_push; - Settings settings_push = task_cluster->settings_push; - - LOG_DEBUG(log, "Execute distributed DROP TABLE: {}", query); - /// We have to drop partition_piece on each replica - UInt64 num_nodes = executeQueryOnCluster( - cluster_push, query, - settings_push, - PoolMode::GET_MANY, - ClusterExecutionMode::ON_EACH_NODE); - - LOG_DEBUG(log, "DROP TABLE query was successfully executed on {} nodes.", toString(num_nodes)); + dropHelpingTablesByPieceNumber(task_table, current_piece_number); } } - void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskTable & task_table, const String & partition_name) { LOG_DEBUG(log, "Try drop partition partition from all helping tables."); @@ -1586,7 +1650,7 @@ void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskT PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); - LOG_DEBUG(log, "DROP PARTITION query was successfully executed on {} nodes.", toString(num_nodes)); + LOG_INFO(log, "DROP PARTITION query was successfully executed on {} nodes.", toString(num_nodes)); } LOG_DEBUG(log, "All helping tables dropped partition {}", partition_name); } diff --git a/programs/copier/ClusterCopier.h b/programs/copier/ClusterCopier.h index 9aff5493cf8..95bb54cf4e1 100644 --- a/programs/copier/ClusterCopier.h +++ b/programs/copier/ClusterCopier.h @@ -123,12 +123,13 @@ protected: bool tryDropPartitionPiece(ShardPartition & task_partition, const size_t current_piece_number, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); - static constexpr UInt64 max_table_tries = 1000; - static constexpr UInt64 max_shard_partition_tries = 600; - static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 100; + static constexpr UInt64 max_table_tries = 3; + static constexpr UInt64 max_shard_partition_tries = 3; + static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 3; bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); + TaskStatus tryCreateDestinationTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); /// Job for copying partition from particular shard. TaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, @@ -149,6 +150,8 @@ protected: void dropHelpingTables(const TaskTable & task_table); + void dropHelpingTablesByPieceNumber(const TaskTable & task_table, size_t current_piece_number); + /// Is used for usage less disk space. /// After all pieces were successfully moved to original destination /// table we can get rid of partition pieces (partitions in helping tables). diff --git a/programs/copier/TaskCluster.h b/programs/copier/TaskCluster.h index 5b28f461dd8..1a50597d07f 100644 --- a/programs/copier/TaskCluster.h +++ b/programs/copier/TaskCluster.h @@ -98,6 +98,7 @@ inline void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfigurat set_default_value(settings_pull.max_block_size, 8192UL); set_default_value(settings_pull.preferred_block_size_bytes, 0); set_default_value(settings_push.insert_distributed_timeout, 0); + set_default_value(settings_push.replication_alter_partitions_sync, 2); } } From 1b1d425e426e0a88ff3911b9946e8725909ab595 Mon Sep 17 00:00:00 2001 From: turbo jason Date: Mon, 22 Mar 2021 10:27:32 +0800 Subject: [PATCH 03/34] remove duplicated code --- programs/copier/ClusterCopier.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 5014f75ae48..a11c47e4e66 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -598,8 +598,6 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t original_table.second + "_piece_" + toString(current_piece_number)); - Settings settings_push = task_cluster->settings_push; - Settings settings_push = task_cluster->settings_push; ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_NODE; UInt64 max_successful_executions_per_shard = 0; From 6aa89d0ee36e2720f95f60eaa24bde37cc812e8c Mon Sep 17 00:00:00 2001 From: turbo jason Date: Tue, 23 Mar 2021 11:00:06 +0800 Subject: [PATCH 04/34] remove duplicated code --- programs/copier/ClusterCopier.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index a11c47e4e66..2d7765f372e 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -1549,7 +1549,6 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( { tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); create_is_dirty_node(clean_state_clock); - dropHelpingTablesByPieceNumber(task_table, current_piece_number); return TaskStatus::Error; } } From f4be05ca25ff51c9eb74bef53f7323e288d6b1b7 Mon Sep 17 00:00:00 2001 From: turbo jason Date: Tue, 23 Mar 2021 15:04:25 +0800 Subject: [PATCH 05/34] fix code style --- programs/copier/ClusterCopier.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 2d7765f372e..83832c50147 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -877,7 +877,7 @@ bool ClusterCopier::tryDropPartitionPiece( bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table) { - //first create destionation table + /// Create destination table TaskStatus task_status = TaskStatus::Error; task_status = tryCreateDestinationTable(timeouts, task_table); From 22c9cb1aa30c12fe4b36c603b5aad49bcbc02772 Mon Sep 17 00:00:00 2001 From: turbo jason Date: Tue, 23 Mar 2021 15:35:15 +0800 Subject: [PATCH 06/34] fix code style --- programs/copier/ClusterCopier.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 83832c50147..cc524b8fc35 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -628,7 +628,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t { LOG_INFO( log, - "Destination tables {} have been executed alter query successfuly on {} shards of {}", + "Destination tables {} have been executed alter query successfully on {} shards of {}", getQuotedTable(task_table.table_push), num_nodes, task_table.cluster_push->getShardCount()); From 1868ca34b8740a4948acbb3c323c7e25889825e3 Mon Sep 17 00:00:00 2001 From: PHO Date: Fri, 19 Mar 2021 10:28:30 +0900 Subject: [PATCH 07/34] Fix build on GCC where std::isinf() cannot be called without the namespace prefix --- src/Dictionaries/PolygonDictionary.cpp | 2 +- src/Functions/geometryConverters.h | 2 +- src/Storages/PostgreSQL/PostgreSQLReplicaConnection.h | 1 + 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index 1d0c75f6bff..04559d701c9 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -271,7 +271,7 @@ std::vector IPolygonDictionary::extractPoints(const C throw Exception(ErrorCodes::BAD_ARGUMENTS, "PolygonDictionary input point component must not be NaN"); - if (isinf(x) || isinf(y)) + if (std::isinf(x) || std::isinf(y)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "PolygonDictionary input point component must not be infinite"); diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 283bb1bb7f4..12f0faf3a70 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -76,7 +76,7 @@ struct ColumnToPointsConverter if (isNaN(first) || isNaN(second)) throw Exception("Point's component must not be NaN", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (isinf(first) || isinf(second)) + if (std::isinf(first) || std::isinf(second)) throw Exception("Point's component must not be infinite", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); answer[i] = Point(first, second); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicaConnection.h b/src/Storages/PostgreSQL/PostgreSQLReplicaConnection.h index f4b2979d116..1ed442873a2 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicaConnection.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicaConnection.h @@ -3,6 +3,7 @@ #include #include #include "PostgreSQLConnectionPool.h" +#include namespace DB From 6606d8ac0b2979e921f7752fb73b9b203a74fb74 Mon Sep 17 00:00:00 2001 From: PHO Date: Fri, 19 Mar 2021 11:41:34 +0900 Subject: [PATCH 08/34] Correct comments about what parseIPv4() and parseIPv6() actually do They were wrong as to what endian their results were in. --- src/Common/formatIPv6.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/formatIPv6.h b/src/Common/formatIPv6.h index bd0c68d70f9..1cb9d3d1d81 100644 --- a/src/Common/formatIPv6.h +++ b/src/Common/formatIPv6.h @@ -25,7 +25,7 @@ void formatIPv6(const unsigned char * src, char *& dst, uint8_t zeroed_tail_byte /** Unsafe (no bounds-checking for src nor dst), optimized version of parsing IPv4 string. * - * Parses the input string `src` and stores binary BE value into buffer pointed by `dst`, + * Parses the input string `src` and stores binary host-endian value into buffer pointed by `dst`, * which should be long enough. * That is "127.0.0.1" becomes 0x7f000001. * @@ -63,7 +63,7 @@ inline bool parseIPv4(const char * src, unsigned char * dst) /** Unsafe (no bounds-checking for src nor dst), optimized version of parsing IPv6 string. * * Slightly altered implementation from http://svn.apache.org/repos/asf/apr/apr/trunk/network_io/unix/inet_pton.c -* Parses the input string `src` and stores binary LE value into buffer pointed by `dst`, +* Parses the input string `src` and stores binary big-endian value into buffer pointed by `dst`, * which should be long enough. In case of failure zeroes * IPV6_BINARY_LENGTH bytes of buffer pointed by `dst`. * From 9ece398c354397ecaa3ad835129b9033db6e4823 Mon Sep 17 00:00:00 2001 From: PHO Date: Fri, 19 Mar 2021 16:24:38 +0900 Subject: [PATCH 09/34] Changed the return type of DB::getCIDRMaskIPv6() It now returns std::array instead of uint8_t *. Also added DB::getCIDRMaskIPv4(). --- src/Common/IPv6ToBinary.cpp | 37 +++++++++++++++++++++------------ src/Common/IPv6ToBinary.h | 9 +++++--- src/Functions/FunctionsCoding.h | 4 ++-- 3 files changed, 32 insertions(+), 18 deletions(-) diff --git a/src/Common/IPv6ToBinary.cpp b/src/Common/IPv6ToBinary.cpp index 3c004a5a84e..54630eda6f3 100644 --- a/src/Common/IPv6ToBinary.cpp +++ b/src/Common/IPv6ToBinary.cpp @@ -13,8 +13,11 @@ namespace DB /// Result array could be indexed with all possible uint8 values without extra check. /// For values greater than 128 we will store same value as for 128 (all bits set). constexpr size_t IPV6_MASKS_COUNT = 256; +using RawMaskArrayV6 = std::array; -using RawMaskArray = std::array; +/// Same for IPv4 +constexpr size_t IPV4_MASKS_COUNT = 256; +using RawMaskArrayV4 = std::array; void IPv6ToRawBinary(const Poco::Net::IPAddress & address, char * res) { @@ -41,33 +44,41 @@ std::array IPv6ToBinary(const Poco::Net::IPAddress & address) return res; } -static constexpr RawMaskArray generateBitMask(size_t prefix) +template +static constexpr RawMaskArrayT generateBitMask(size_t prefix) { - if (prefix >= 128) - prefix = 128; - RawMaskArray arr{0}; + RawMaskArrayT arr{0}; + if (prefix >= arr.size() * 8) + prefix = arr.size() * 8; size_t i = 0; for (; prefix >= 8; ++i, prefix -= 8) arr[i] = 0xff; if (prefix > 0) arr[i++] = ~(0xff >> prefix); - while (i < 16) + while (i < arr.size()) arr[i++] = 0x00; return arr; } -static constexpr std::array generateBitMasks() +template +static constexpr std::array generateBitMasks() { - std::array arr{}; - for (size_t i = 0; i < IPV6_MASKS_COUNT; ++i) - arr[i] = generateBitMask(i); + std::array arr{}; + for (size_t i = 0; i < masksCount; ++i) + arr[i] = generateBitMask(i); return arr; } -const uint8_t * getCIDRMaskIPv6(UInt8 prefix_len) +const std::array & getCIDRMaskIPv6(UInt8 prefix_len) { - static constexpr std::array IPV6_RAW_MASK_ARRAY = generateBitMasks(); - return IPV6_RAW_MASK_ARRAY[prefix_len].data(); + static constexpr auto IPV6_RAW_MASK_ARRAY = generateBitMasks(); + return IPV6_RAW_MASK_ARRAY[prefix_len]; +} + +const std::array & getCIDRMaskIPv4(UInt8 prefix_len) +{ + static constexpr auto IPV4_RAW_MASK_ARRAY = generateBitMasks(); + return IPV4_RAW_MASK_ARRAY[prefix_len]; } } diff --git a/src/Common/IPv6ToBinary.h b/src/Common/IPv6ToBinary.h index 2e47238aeba..6d2d3d33e97 100644 --- a/src/Common/IPv6ToBinary.h +++ b/src/Common/IPv6ToBinary.h @@ -14,9 +14,12 @@ void IPv6ToRawBinary(const Poco::Net::IPAddress & address, char * res); /// Convert IP address to 16-byte array with IPv6 data (big endian). If it's an IPv4, map it to IPv6. std::array IPv6ToBinary(const Poco::Net::IPAddress & address); -/// Returns pointer to 16-byte array containing mask with first `prefix_len` bits set to `1` and `128 - prefix_len` to `0`. -/// Pointer is valid during all program execution time and doesn't require freeing. +/// Returns a reference to 16-byte array containing mask with first `prefix_len` bits set to `1` and `128 - prefix_len` to `0`. +/// The reference is valid during all program execution time. /// Values of prefix_len greater than 128 interpreted as 128 exactly. -const uint8_t * getCIDRMaskIPv6(UInt8 prefix_len); +const std::array & getCIDRMaskIPv6(UInt8 prefix_len); + +/// This is identical to getCIDRMaskIPv6 except it's for IPv4 addresses. +const std::array & getCIDRMaskIPv4(UInt8 prefix_len); } diff --git a/src/Functions/FunctionsCoding.h b/src/Functions/FunctionsCoding.h index abfb1e83a77..ee15561f133 100644 --- a/src/Functions/FunctionsCoding.h +++ b/src/Functions/FunctionsCoding.h @@ -1645,7 +1645,7 @@ private: static inline void applyCIDRMask(const UInt8 * __restrict src, UInt8 * __restrict dst_lower, UInt8 * __restrict dst_upper, UInt8 bits_to_keep) { - __m128i mask = _mm_loadu_si128(reinterpret_cast(getCIDRMaskIPv6(bits_to_keep))); + __m128i mask = _mm_loadu_si128(reinterpret_cast(getCIDRMaskIPv6(bits_to_keep).data())); __m128i lower = _mm_and_si128(_mm_loadu_si128(reinterpret_cast(src)), mask); _mm_storeu_si128(reinterpret_cast<__m128i *>(dst_lower), lower); @@ -1659,7 +1659,7 @@ private: /// NOTE IPv6 is stored in memory in big endian format that makes some difficulties. static void applyCIDRMask(const UInt8 * __restrict src, UInt8 * __restrict dst_lower, UInt8 * __restrict dst_upper, UInt8 bits_to_keep) { - const auto * mask = getCIDRMaskIPv6(bits_to_keep); + const auto & mask = getCIDRMaskIPv6(bits_to_keep); for (size_t i = 0; i < 16; ++i) { From 26dc62936646860d802fc84118dd4a7e9ed9d1db Mon Sep 17 00:00:00 2001 From: PHO Date: Wed, 17 Feb 2021 17:57:48 +0900 Subject: [PATCH 10/34] Add a function isIPAddressContainedIn() This function takes an IP address and a CIDR network prefix, and determines if the address is contained in the network. It supports both IPv4 and IPv6 addresses, and expects arguments to be strings. ClickHouse doesn't support union types. We cannot use the native IPv4 and IPv6 types for this reason. --- .../functions/ip-address-functions.md | 52 ++ src/Functions/isIPAddressContainedIn.cpp | 621 ++++++++++++++++++ .../registerFunctionsMiscellaneous.cpp | 2 + src/Functions/ya.make | 1 + .../01700_isIPAddressContainedIn.reference | 76 +++ .../01700_isIPAddressContainedIn.sql | 86 +++ 6 files changed, 838 insertions(+) create mode 100644 src/Functions/isIPAddressContainedIn.cpp create mode 100644 tests/queries/0_stateless/01700_isIPAddressContainedIn.reference create mode 100644 tests/queries/0_stateless/01700_isIPAddressContainedIn.sql diff --git a/docs/en/sql-reference/functions/ip-address-functions.md b/docs/en/sql-reference/functions/ip-address-functions.md index a24cd464946..b293cd2d4a7 100644 --- a/docs/en/sql-reference/functions/ip-address-functions.md +++ b/docs/en/sql-reference/functions/ip-address-functions.md @@ -394,3 +394,55 @@ Result: └──────────────────┴────────────────────┘ ``` +## isIPAddressContainedIn {#isipaddresscontainedin} + +Determines if an IP address is contained in a network represented in the [CIDR](https://en.wikipedia.org/wiki/Classless_Inter-Domain_Routing) notation. Returns `1` if true, or `0` otherwise. + +**Syntax** + +``` sql +isIPAddressContainedIn(address, prefix) +``` + +This function accepts both IPv4 and IPv6 addresses (and networks) represented as strings. It returns `0` if the IP version of the address and the CIDR don't match. It returns `NULL` if either of the arguments is `NULL`. + +**Parameters** + +- `address` — An IPv4 or IPv6 address. [String](../../sql-reference/data-types/string.md). +- `prefix` — An IPv4 or IPv6 network prefix in CIDR. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- `1` or `0`. + +Type: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +``` sql +SELECT isIPAddressContainedIn('127.0.0.1', '127.0.0.0/8') +``` + +Result: + +``` text +┌─isIPAddressContainedIn('127.0.0.1', '127.0.0.0/8')─┐ +│ 1 │ +└────────────────────────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT isIPAddressContainedIn('127.0.0.1', 'ffff::/16') +``` + +Result: + +``` text +┌─isIPAddressContainedIn('127.0.0.1', 'ffff::/16')─┐ +│ 0 │ +└──────────────────────────────────────────────────┘ +``` diff --git a/src/Functions/isIPAddressContainedIn.cpp b/src/Functions/isIPAddressContainedIn.cpp new file mode 100644 index 00000000000..49a0c5b74de --- /dev/null +++ b/src/Functions/isIPAddressContainedIn.cpp @@ -0,0 +1,621 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + namespace ErrorCodes + { + extern const int CANNOT_PARSE_TEXT; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + } +} + +namespace ipaddr +{ + class Address + { + public: + Address() = delete; + + explicit Address(const StringRef & in) + { + // IP address parser functions require that the input is + // NULL-terminated so we need to copy it. + const auto in_copy = std::string(in); + + UInt32 v4; + if (DB::parseIPv4(in_copy.c_str(), reinterpret_cast(&v4))) + { + addr = V4(v4); + } + else + { + V6 v6; + if (DB::parseIPv6(in_copy.c_str(), v6.addr.data())) + addr = std::move(v6); + else + throw DB::Exception("Neither IPv4 nor IPv6 address: " + in_copy, + DB::ErrorCodes::CANNOT_PARSE_TEXT); + } + } + + template + struct IPVersionBase + { + IPVersionBase() + : addr {} {} + + explicit IPVersionBase(const std::array & octets) + : addr(octets) {} + + constexpr size_t numBits() const + { + return numOctets * 8; + } + + uint8_t operator[] (size_t i) const + { + assert(i >= 0 && i < numOctets); + return addr[i]; + } + + uint8_t & operator[] (size_t i) + { + assert(i >= 0 && i < numOctets); + return addr[i]; + } + + bool operator<= (const ConcreteType & rhs) const + { + for (size_t i = 0; i < numOctets; i++) + { + if ((*this)[i] < rhs[i]) return true; + if ((*this)[i] > rhs[i]) return false; + } + return true; + } + + bool operator>= (const ConcreteType & rhs) const + { + for (size_t i = 0; i < numOctets; i++) + { + if ((*this)[i] > rhs[i]) return true; + if ((*this)[i] < rhs[i]) return false; + } + return true; + } + + ConcreteType operator& (const ConcreteType & rhs) const + { + ConcreteType lhs(addr); + + for (size_t i = 0; i < numOctets; i++) + lhs[i] &= rhs[i]; + + return lhs; + } + + ConcreteType operator| (const ConcreteType & rhs) const + { + ConcreteType lhs(addr); + + for (size_t i = 0; i < numOctets; i++) + lhs[i] |= rhs[i]; + + return lhs; + } + + ConcreteType operator~ () const + { + ConcreteType tmp(addr); + + for (size_t i = 0; i < numOctets; i++) + tmp[i] = ~tmp[i]; + + return tmp; + } + + private: + // Big-endian + std::array addr; + friend class Address; + }; + + struct V4 : public IPVersionBase + { + V4() = default; + + explicit V4(UInt32 addr_) + { + addr[0] = (addr_ >> 24) & 0xFF; + addr[1] = (addr_ >> 16) & 0xFF; + addr[2] = (addr_ >> 8) & 0xFF; + addr[3] = addr_ & 0xFF; + } + + explicit V4(const std::array & components) + : IPVersionBase(components) {} + }; + + struct V6 : public IPVersionBase + { + V6() = default; + + explicit V6(const std::array & components) + : IPVersionBase(components) {} + }; + + constexpr const std::variant & variant() const + { + return addr; + } + + private: + std::variant addr; + }; + + class CIDR + { + public: + CIDR() = delete; + + explicit CIDR(const StringRef & in) + { + const auto in_view = std::string_view(in); + const auto pos_slash = in_view.find('/'); + + if (pos_slash == std::string_view::npos) + throw DB::Exception("The text does not contain '/': " + std::string(in_view), + DB::ErrorCodes::CANNOT_PARSE_TEXT); + + prefix = Address(StringRef(in_view.substr(0, pos_slash))); + + // DB::parse() in ignores + // non-digit characters. std::stoi() skips whitespaces. We + // need to parse the prefix bits in a strict way. + + if (pos_slash + 1 == in_view.size()) + throw DB::Exception("The CIDR has no prefix bits: " + std::string(in_view), + DB::ErrorCodes::CANNOT_PARSE_TEXT); + + bits = 0; + for (size_t i = pos_slash + 1; i < in_view.size(); i++) + { + const auto c = in_view[i]; + if (c >= '0' && c <= '9') + { + bits *= 10; + bits += c - '0'; + } + else + { + throw DB::Exception("The CIDR has a malformed prefix bits: " + std::string(in_view), + DB::ErrorCodes::CANNOT_PARSE_TEXT); + } + } + + const size_t max_bits + = std::visit([&](const auto & addr_v) -> size_t + { + return addr_v.numBits(); + }, prefix->variant()); + if (bits > max_bits) + throw DB::Exception("The CIDR has an invalid prefix bits: " + std::string(in_view), + DB::ErrorCodes::CANNOT_PARSE_TEXT); + } + + private: + template + static PrefixT toMask(uint8_t bits) + { + if constexpr (std::is_same_v) + { + return PrefixT(DB::getCIDRMaskIPv4(bits)); + } + else + { + return PrefixT(DB::getCIDRMaskIPv6(bits)); + } + } + + template + static PrefixT startOf(const PrefixT & prefix, uint8_t bits) + { + return prefix & toMask(bits); + } + + template + static PrefixT endOf(const PrefixT & prefix, uint8_t bits) + { + return prefix | ~toMask(bits); + } + + /* Convert a CIDR notation into an IP address range [start, end]. */ + template + static std::pair toRange(const PrefixT & prefix, uint8_t bits) + { + return std::make_pair(startOf(prefix, bits), endOf(prefix, bits)); + } + + public: + bool contains(const Address & addr) const + { + return std::visit([&](const auto & addr_v) -> bool + { + return std::visit([&](const auto & prefix_v) -> bool + { + using AddrT = std::decay_t; + using PrefixT = std::decay_t; + + if constexpr (std::is_same_v) + { + if constexpr (std::is_same_v) + { + const auto range = toRange(prefix_v, bits); + return addr_v >= range.first && addr_v <= range.second; + } + else + { + return false; // IP version mismatch is not an error. + } + } + else + { + if constexpr (std::is_same_v) + { + const auto range = toRange(prefix_v, bits); + return addr_v >= range.first && addr_v <= range.second; + } + else + { + return false; // IP version mismatch is not an error. + } + } + }, prefix->variant()); + }, addr.variant()); + } + + private: + std::optional
prefix; // Guaranteed to have a value after construction. + uint8_t bits; + }; +} + +namespace DB +{ + template + class ExecutableFunctionIsIPAddressContainedIn : public IExecutableFunctionImpl + { + public: + String getName() const override + { + return Name::name; + } + + ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type, size_t input_rows_count) const override + { + const IColumn * col_addr = arguments[0].column.get(); + const IColumn * col_cidr = arguments[1].column.get(); + + if (const auto * col_addr_const = checkAndGetAnyColumnConst(col_addr)) + { + // col_addr_const is constant and is either String or + // Nullable(String). We don't care which one it exactly is. + + if (const auto * col_cidr_const = checkAndGetAnyColumnConst(col_cidr)) + return executeImpl(*col_addr_const, *col_cidr_const, return_type, input_rows_count); + else + return executeImpl(*col_addr_const, *col_cidr, return_type, input_rows_count); + } + else + { + if (const auto * col_cidr_const = checkAndGetAnyColumnConst(col_cidr)) + return executeImpl(*col_addr, *col_cidr_const, return_type, input_rows_count); + else + return executeImpl(*col_addr, *col_cidr, input_rows_count); + } + } + + bool useDefaultImplementationForNulls() const override + { + // We can't use the default implementation because that would end up + // parsing invalid addresses or prefixes at NULL fields, which would + // throw exceptions instead of returning NULL. + return false; + } + + private: + // Like checkAndGetColumnConst() but this function doesn't + // care about the type of data column. + static const ColumnConst * checkAndGetAnyColumnConst(const IColumn * column) + { + if (!column || !isColumnConst(*column)) + return nullptr; + + return assert_cast(column); + } + + // Both columns are constant. + ColumnPtr executeImpl(const ColumnConst & col_addr_const, const ColumnConst & col_cidr_const, const DataTypePtr & return_type, size_t input_rows_count) const + { + const auto & col_addr = col_addr_const.getDataColumn(); + const auto & col_cidr = col_cidr_const.getDataColumn(); + + if (col_addr.isNullAt(0) || col_cidr.isNullAt(0)) + { + // If either of the arguments are NULL, the result is also NULL. + assert(return_type->isNullable()); + return return_type->createColumnConstWithDefaultValue(input_rows_count); + } + else + { + const auto addr = ipaddr::Address(col_addr.getDataAt(0)); + const auto cidr = ipaddr::CIDR(col_cidr.getDataAt(0)); + + ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(1); + ColumnUInt8::Container & vec_res = col_res->getData(); + + vec_res[0] = cidr.contains(addr) ? 1 : 0; + + if (return_type->isNullable()) + { + ColumnUInt8::MutablePtr col_null_map_res = ColumnUInt8::create(1); + ColumnUInt8::Container & vec_null_map_res = col_null_map_res->getData(); + + vec_null_map_res[0] = false; + + return ColumnConst::create(ColumnNullable::create(std::move(col_res), std::move(col_null_map_res)), input_rows_count); + } + else + { + return ColumnConst::create(std::move(col_res), input_rows_count); + } + } + } + + // Address is constant. + ColumnPtr executeImpl(const ColumnConst & col_addr_const, const IColumn & col_cidr, const DataTypePtr & return_type, size_t input_rows_count) const + { + const auto & col_addr = col_addr_const.getDataColumn(); + + if (col_addr.isNullAt(0)) + { + // It's constant NULL so the result is also constant NULL. + assert(return_type->isNullable()); + return return_type->createColumnConstWithDefaultValue(input_rows_count); + } + else + { + const auto addr = ipaddr::Address(col_addr.getDataAt(0)); + + ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); + ColumnUInt8::Container & vec_res = col_res->getData(); + + if (col_addr.isNullable() || col_cidr.isNullable()) + { + ColumnUInt8::MutablePtr col_null_map_res = ColumnUInt8::create(input_rows_count); + ColumnUInt8::Container & vec_null_map_res = col_null_map_res->getData(); + + for (size_t i = 0; i < input_rows_count; i++) + { + if (col_cidr.isNullAt(i)) + { + vec_null_map_res[i] = true; + } + else + { + const auto cidr = ipaddr::CIDR(col_cidr.getDataAt(i)); + vec_res[i] = cidr.contains(addr) ? 1 : 0; + vec_null_map_res[i] = false; + } + } + + return ColumnNullable::create(std::move(col_res), std::move(col_null_map_res)); + } + else + { + for (size_t i = 0; i < input_rows_count; i++) + { + const auto cidr = ipaddr::CIDR(col_cidr.getDataAt(i)); + vec_res[i] = cidr.contains(addr) ? 1 : 0; + } + + return col_res; + } + } + } + + // CIDR is constant. + ColumnPtr executeImpl(const IColumn & col_addr, const ColumnConst & col_cidr_const, const DataTypePtr & return_type, size_t input_rows_count) const + { + const auto & col_cidr = col_cidr_const.getDataColumn(); + + if (col_cidr.isNullAt(0)) + { + // It's constant NULL so the result is also constant NULL. + assert(return_type->isNullable()); + return return_type->createColumnConstWithDefaultValue(input_rows_count); + } + else + { + const auto cidr = ipaddr::CIDR(col_cidr.getDataAt(0)); + + ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); + ColumnUInt8::Container & vec_res = col_res->getData(); + + if (col_addr.isNullable() || col_cidr.isNullable()) + { + ColumnUInt8::MutablePtr col_null_map_res = ColumnUInt8::create(input_rows_count); + ColumnUInt8::Container & vec_null_map_res = col_null_map_res->getData(); + + for (size_t i = 0; i < input_rows_count; i++) + { + if (col_addr.isNullAt(i)) + { + vec_null_map_res[i] = true; + } + else + { + const auto addr = ipaddr::Address(col_addr.getDataAt(i)); + vec_res[i] = cidr.contains(addr) ? 1 : 0; + vec_null_map_res[i] = false; + } + } + + return ColumnNullable::create(std::move(col_res), std::move(col_null_map_res)); + } + else + { + for (size_t i = 0; i < input_rows_count; i++) + { + const auto addr = ipaddr::Address(col_addr.getDataAt(i)); + vec_res[i] = cidr.contains(addr) ? 1 : 0; + } + + return col_res; + } + } + } + + // Neither are constant. + ColumnPtr executeImpl(const IColumn & col_addr, const IColumn & col_cidr, size_t input_rows_count) const + { + ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); + ColumnUInt8::Container & vec_res = col_res->getData(); + + if (col_addr.isNullable() || col_cidr.isNullable()) + { + ColumnUInt8::MutablePtr col_null_map_res = ColumnUInt8::create(input_rows_count); + ColumnUInt8::Container & vec_null_map_res = col_null_map_res->getData(); + + for (size_t i = 0; i < input_rows_count; i++) + { + if (col_addr.isNullAt(i) || col_cidr.isNullAt(i)) + { + vec_null_map_res[i] = true; + } + else + { + const auto addr = ipaddr::Address(col_addr.getDataAt(i)); + const auto cidr = ipaddr::CIDR(col_cidr.getDataAt(i)); + + vec_res[i] = cidr.contains(addr) ? 1 : 0; + vec_null_map_res[i] = false; + } + } + + return ColumnNullable::create(std::move(col_res), std::move(col_null_map_res)); + } + else + { + for (size_t i = 0; i < input_rows_count; i++) + { + const auto addr = ipaddr::Address(col_addr.getDataAt(i)); + const auto cidr = ipaddr::CIDR(col_cidr.getDataAt(i)); + + vec_res[i] = cidr.contains(addr) ? 1 : 0; + } + + return col_res; + } + } + }; + + template + class FunctionBaseIsIPAddressContainedIn : public IFunctionBaseImpl + { + public: + explicit FunctionBaseIsIPAddressContainedIn(DataTypes argument_types_, DataTypePtr return_type_) + : argument_types(std::move(argument_types_)) + , return_type(std::move(return_type_)) {} + + String getName() const override + { + return Name::name; + } + + const DataTypes & getArgumentTypes() const override + { + return argument_types; + } + + const DataTypePtr & getResultType() const override + { + return return_type; + } + + ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName &) const override + { + return std::make_unique>(); + } + + private: + DataTypes argument_types; + DataTypePtr return_type; + }; + + template + class IsIPAddressContainedInOverloadResolver : public IFunctionOverloadResolverImpl + { + public: + static constexpr auto name = Name::name; + + static FunctionOverloadResolverImplPtr create(const Context &) + { + return std::make_unique>(); + } + + String getName() const override + { + return Name::name; + } + + FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + const DataTypePtr & addr_type = removeNullable(arguments[0].type); + const DataTypePtr & prefix_type = removeNullable(arguments[1].type); + DataTypes argument_types = { addr_type, prefix_type }; + + /* The arguments can be any of Nullable(NULL), Nullable(String), and + * String. We can't do this check in getReturnType() because it + * won't be called when there are any constant NULLs in the + * arguments. */ + if (!(WhichDataType(addr_type).isNothing() || isString(addr_type)) || + !(WhichDataType(prefix_type).isNothing() || isString(prefix_type))) + throw Exception("The arguments of function " + getName() + " must be String", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_unique>(argument_types, return_type); + } + + DataTypePtr getReturnType(const DataTypes &) const override + { + return std::make_shared(); + } + + size_t getNumberOfArguments() const override + { + return 2; + } + }; + + struct NameIsIPAddressContainedIn + { + static constexpr auto name = "isIPAddressContainedIn"; + }; + + void registerFunctionIsIPAddressContainedIn(FunctionFactory & factory) + { + factory.registerFunction>(); + } +} diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index ca9bc32486e..1e2a7dad3ec 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -71,6 +71,7 @@ void registerFunctionTcpPort(FunctionFactory &); void registerFunctionByteSize(FunctionFactory &); void registerFunctionFile(FunctionFactory & factory); void registerFunctionConnectionID(FunctionFactory & factory); +void registerFunctionIsIPAddressContainedIn(FunctionFactory &); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -142,6 +143,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionByteSize(factory); registerFunctionFile(factory); registerFunctionConnectionID(factory); + registerFunctionIsIPAddressContainedIn(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/src/Functions/ya.make b/src/Functions/ya.make index aed2bd9b70d..0d2ca6e02ea 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -309,6 +309,7 @@ SRCS( isConstant.cpp isDecimalOverflow.cpp isFinite.cpp + isIPAddressContainedIn.cpp isInfinite.cpp isNaN.cpp isNotNull.cpp diff --git a/tests/queries/0_stateless/01700_isIPAddressContainedIn.reference b/tests/queries/0_stateless/01700_isIPAddressContainedIn.reference new file mode 100644 index 00000000000..98650dd69f4 --- /dev/null +++ b/tests/queries/0_stateless/01700_isIPAddressContainedIn.reference @@ -0,0 +1,76 @@ +# Invocation with constants +1 +0 +1 +0 +# Invocation with non-constant addresses +192.168.99.255 192.168.100.0/22 0 +192.168.100.1 192.168.100.0/22 1 +192.168.103.255 192.168.100.0/22 1 +192.168.104.0 192.168.100.0/22 0 +::192.168.99.255 ::192.168.100.0/118 0 +::192.168.100.1 ::192.168.100.0/118 1 +::192.168.103.255 ::192.168.100.0/118 1 +::192.168.104.0 ::192.168.100.0/118 0 +# Invocation with non-constant prefixes +192.168.100.1 192.168.100.0/22 1 +192.168.100.1 192.168.100.0/24 1 +192.168.100.1 192.168.100.0/32 0 +::192.168.100.1 ::192.168.100.0/118 1 +::192.168.100.1 ::192.168.100.0/120 1 +::192.168.100.1 ::192.168.100.0/128 0 +# Invocation with non-constants +192.168.100.1 192.168.100.0/22 1 +192.168.100.1 192.168.100.0/24 1 +192.168.103.255 192.168.100.0/22 1 +192.168.103.255 192.168.100.0/24 0 +::192.168.100.1 ::192.168.100.0/118 1 +::192.168.100.1 ::192.168.100.0/120 1 +::192.168.103.255 ::192.168.100.0/118 1 +::192.168.103.255 ::192.168.100.0/120 0 +# Mismatching IP versions is not an error. +0 +0 +0 +0 +# Arguments can be nullable. +## Nullable address +\N +\N +1 +## Nullable prefix +\N +\N +1 +## Both nullable +\N +\N +1 +# Non-constant nullable arguments +## Non-constant address +127.0.0.1 127.0.0.0/8 1 +\N 127.0.0.0/8 \N +127.0.0.1 \N \N +\N \N \N +127.0.0.1 \N \N +\N \N \N +127.0.0.1 127.0.0.0/8 1 +\N 127.0.0.0/8 \N +127.0.0.1 127.0.0.0/8 1 +## Non-constant prefix +127.0.0.1 127.0.0.0/8 1 +127.0.0.1 \N \N +\N 127.0.0.0/8 \N +\N \N \N +\N 127.0.0.0/8 \N +\N \N \N +127.0.0.1 127.0.0.0/8 1 +127.0.0.1 \N \N +127.0.0.1 127.0.0.0/8 1 +## Both non-constant +127.0.0.1 127.0.0.0/8 1 +127.0.0.1 \N \N +\N 127.0.0.0/8 \N +\N \N \N +# Unparsable arguments +# Wrong argument types diff --git a/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql b/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql new file mode 100644 index 00000000000..0618aa0cfa5 --- /dev/null +++ b/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql @@ -0,0 +1,86 @@ +-- +SELECT '# Invocation with constants'; + +SELECT isIPAddressContainedIn('127.0.0.1', '127.0.0.0/8'); +SELECT isIPAddressContainedIn('128.0.0.1', '127.0.0.0/8'); + +SELECT isIPAddressContainedIn('ffff::1', 'ffff::/16'); +SELECT isIPAddressContainedIn('fffe::1', 'ffff::/16'); + +-- +SELECT '# Invocation with non-constant addresses'; + +WITH arrayJoin(['192.168.99.255', '192.168.100.1', '192.168.103.255', '192.168.104.0']) as addr, '192.168.100.0/22' as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH arrayJoin(['::192.168.99.255', '::192.168.100.1', '::192.168.103.255', '::192.168.104.0']) as addr, '::192.168.100.0/118' as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); + +-- +SELECT '# Invocation with non-constant prefixes'; + +WITH '192.168.100.1' as addr, arrayJoin(['192.168.100.0/22', '192.168.100.0/24', '192.168.100.0/32']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH '::192.168.100.1' as addr, arrayJoin(['::192.168.100.0/118', '::192.168.100.0/120', '::192.168.100.0/128']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); + +-- +SELECT '# Invocation with non-constants'; + +WITH arrayJoin(['192.168.100.1', '192.168.103.255']) as addr, arrayJoin(['192.168.100.0/22', '192.168.100.0/24']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH arrayJoin(['::192.168.100.1', '::192.168.103.255']) as addr, arrayJoin(['::192.168.100.0/118', '::192.168.100.0/120']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); + +-- +SELECT '# Mismatching IP versions is not an error.'; + +SELECT isIPAddressContainedIn('127.0.0.1', 'ffff::/16'); +SELECT isIPAddressContainedIn('127.0.0.1', '::127.0.0.1/128'); +SELECT isIPAddressContainedIn('::1', '127.0.0.0/8'); +SELECT isIPAddressContainedIn('::127.0.0.1', '127.0.0.1/32'); + +-- +SELECT '# Arguments can be nullable.'; + +SELECT '## Nullable address'; +SELECT isIPAddressContainedIn(NULL , '127.0.0.0/8'); +SELECT isIPAddressContainedIn(CAST(NULL, 'Nullable(String)') , '127.0.0.0/8'); +SELECT isIPAddressContainedIn(CAST('127.0.0.1', 'Nullable(String)'), '127.0.0.0/8'); + +SELECT '## Nullable prefix'; +SELECT isIPAddressContainedIn('127.0.0.1', NULL); +SELECT isIPAddressContainedIn('127.0.0.1', CAST(NULL, 'Nullable(String)')); +SELECT isIPAddressContainedIn('127.0.0.1', CAST('127.0.0.0/8', 'Nullable(String)')); + +SELECT '## Both nullable'; +SELECT isIPAddressContainedIn(NULL , NULL); +SELECT isIPAddressContainedIn(CAST(NULL, 'Nullable(String)') , CAST(NULL, 'Nullable(String)')); +SELECT isIPAddressContainedIn(CAST('127.0.0.1', 'Nullable(String)'), CAST('127.0.0.0/8', 'Nullable(String)')); + +-- +SELECT '# Non-constant nullable arguments'; + +SELECT '## Non-constant address'; +WITH arrayJoin(['127.0.0.1', NULL]) as addr, '127.0.0.0/8' as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH arrayJoin(['127.0.0.1', NULL]) as addr, NULL as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH arrayJoin(['127.0.0.1', NULL]) as addr, CAST(NULL, 'Nullable(String)') as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH arrayJoin(['127.0.0.1', NULL]) as addr, CAST('127.0.0.0/8', 'Nullable(String)') as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH arrayJoin(['127.0.0.1']) as addr, CAST('127.0.0.0/8', 'Nullable(String)') as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); + +SELECT '## Non-constant prefix'; +WITH '127.0.0.1' as addr, arrayJoin(['127.0.0.0/8', NULL]) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH NULL as addr, arrayJoin(['127.0.0.0/8', NULL]) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH CAST(NULL, 'Nullable(String)') as addr, arrayJoin(['127.0.0.0/8', NULL]) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH CAST('127.0.0.1', 'Nullable(String)') as addr, arrayJoin(['127.0.0.0/8', NULL]) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH CAST('127.0.0.1', 'Nullable(String)') as addr, arrayJoin(['127.0.0.0/8']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); + +SELECT '## Both non-constant'; +WITH arrayJoin(['127.0.0.1', NULL]) as addr, arrayJoin(['127.0.0.0/8', NULL]) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); + +-- +SELECT '# Unparsable arguments'; + +SELECT isIPAddressContainedIn('unparsable', '127.0.0.0/8'); -- { serverError 6 } +SELECT isIPAddressContainedIn('127.0.0.1', 'unparsable'); -- { serverError 6 } + +-- +SELECT '# Wrong argument types'; + +SELECT isIPAddressContainedIn(100, '127.0.0.0/8'); -- { serverError 43 } +SELECT isIPAddressContainedIn('127.0.0.1', 100); -- { serverError 43 } +SELECT isIPAddressContainedIn(100, NULL); -- { serverError 43 } +WITH arrayJoin([NULL, NULL, NULL, NULL]) AS prefix SELECT isIPAddressContainedIn([NULL, NULL, 0, 255, 0], prefix); -- { serverError 43 } From 24aa25d7dcb982079713394078c621c2c671695c Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Mar 2021 12:04:05 +0300 Subject: [PATCH 11/34] Reuse some functions for IPAddressContainedIn --- src/Common/IPv6ToBinary.cpp | 55 +++- src/Common/IPv6ToBinary.h | 5 +- src/Dictionaries/IPAddressDictionary.cpp | 57 +--- src/Functions/isIPAddressContainedIn.cpp | 380 +++++++---------------- 4 files changed, 157 insertions(+), 340 deletions(-) diff --git a/src/Common/IPv6ToBinary.cpp b/src/Common/IPv6ToBinary.cpp index 54630eda6f3..a5ceaa72022 100644 --- a/src/Common/IPv6ToBinary.cpp +++ b/src/Common/IPv6ToBinary.cpp @@ -15,10 +15,6 @@ namespace DB constexpr size_t IPV6_MASKS_COUNT = 256; using RawMaskArrayV6 = std::array; -/// Same for IPv4 -constexpr size_t IPV4_MASKS_COUNT = 256; -using RawMaskArrayV4 = std::array; - void IPv6ToRawBinary(const Poco::Net::IPAddress & address, char * res) { if (Poco::Net::IPAddress::IPv6 == address.family()) @@ -75,10 +71,55 @@ const std::array & getCIDRMaskIPv6(UInt8 prefix_len) return IPV6_RAW_MASK_ARRAY[prefix_len]; } -const std::array & getCIDRMaskIPv4(UInt8 prefix_len) +bool matchIPv4Subnet(UInt32 addr, UInt32 cidr_addr, UInt8 prefix) { - static constexpr auto IPV4_RAW_MASK_ARRAY = generateBitMasks(); - return IPV4_RAW_MASK_ARRAY[prefix_len]; + UInt32 mask = (prefix >= 32) ? 0xffffffffu : ~(0xffffffffu >> prefix); + return (addr & mask) == (cidr_addr & mask); } +#if defined(__SSE2__) +#include + +bool matchIPv6Subnet(const uint8_t * addr, const uint8_t * cidr_addr, UInt8 prefix) +{ + uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8( + _mm_loadu_si128(reinterpret_cast(addr)), + _mm_loadu_si128(reinterpret_cast(cidr_addr)))); + mask = ~mask; + + if (mask) + { + auto offset = __builtin_ctz(mask); + + if (prefix / 8 != offset) + return prefix / 8 < offset; + + auto cmpmask = ~(0xff >> (prefix % 8)); + return (addr[offset] & cmpmask) == (cidr_addr[offset] & cmpmask); + } + return true; +} + +# else + +bool matchIPv6Subnet(const uint8_t * addr, const uint8_t * cidr_addr, UInt8 prefix) +{ + if (prefix > IPV6_BINARY_LENGTH * 8U) + prefix = IPV6_BINARY_LENGTH * 8U; + + size_t i = 0; + for (; prefix >= 8; ++i, prefix -= 8) + { + if (target[i] != cidr_addr[i]) + return false; + } + if (prefix == 0) + return true; + + auto mask = ~(0xff >> prefix); + return (addr[i] & mask) == (cidr_addr[i] & mask); +} + +#endif // __SSE2__ + } diff --git a/src/Common/IPv6ToBinary.h b/src/Common/IPv6ToBinary.h index 6d2d3d33e97..d766d408359 100644 --- a/src/Common/IPv6ToBinary.h +++ b/src/Common/IPv6ToBinary.h @@ -19,7 +19,8 @@ std::array IPv6ToBinary(const Poco::Net::IPAddress & address); /// Values of prefix_len greater than 128 interpreted as 128 exactly. const std::array & getCIDRMaskIPv6(UInt8 prefix_len); -/// This is identical to getCIDRMaskIPv6 except it's for IPv4 addresses. -const std::array & getCIDRMaskIPv4(UInt8 prefix_len); +/// Check that address contained in CIDR range +bool matchIPv4Subnet(UInt32 addr, UInt32 cidr_addr, UInt8 prefix); +bool matchIPv6Subnet(const uint8_t * addr, const uint8_t * cidr_addr, UInt8 prefix); } diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 165fa3a000d..4b51d94f0d8 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -4,19 +4,17 @@ #include #include #include -#include #include #include #include #include -#include #include #include #include #include #include -#include "DictionaryBlockInputStream.h" -#include "DictionaryFactory.h" +#include +#include #include namespace DB @@ -191,57 +189,6 @@ inline static void mapIPv4ToIPv6(UInt32 addr, uint8_t * buf) memcpy(&buf[12], &addr, 4); } -static bool matchIPv4Subnet(UInt32 target, UInt32 addr, UInt8 prefix) -{ - UInt32 mask = (prefix >= 32) ? 0xffffffffu : ~(0xffffffffu >> prefix); - return (target & mask) == addr; -} - -#if defined(__SSE2__) -#include - -static bool matchIPv6Subnet(const uint8_t * target, const uint8_t * addr, UInt8 prefix) -{ - uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8( - _mm_loadu_si128(reinterpret_cast(target)), - _mm_loadu_si128(reinterpret_cast(addr)))); - mask = ~mask; - - if (mask) - { - auto offset = __builtin_ctz(mask); - - if (prefix / 8 != offset) - return prefix / 8 < offset; - - auto cmpmask = ~(0xff >> (prefix % 8)); - return (target[offset] & cmpmask) == addr[offset]; - } - return true; -} - -# else - -static bool matchIPv6Subnet(const uint8_t * target, const uint8_t * addr, UInt8 prefix) -{ - if (prefix > IPV6_BINARY_LENGTH * 8U) - prefix = IPV6_BINARY_LENGTH * 8U; - - size_t i = 0; - for (; prefix >= 8; ++i, prefix -= 8) - { - if (target[i] != addr[i]) - return false; - } - if (prefix == 0) - return true; - - auto mask = ~(0xff >> prefix); - return (target[i] & mask) == addr[i]; -} - -#endif // __SSE2__ - IPAddressDictionary::IPAddressDictionary( const StorageID & dict_id_, const DictionaryStructure & dict_struct_, diff --git a/src/Functions/isIPAddressContainedIn.cpp b/src/Functions/isIPAddressContainedIn.cpp index 49a0c5b74de..45e2f15ed66 100644 --- a/src/Functions/isIPAddressContainedIn.cpp +++ b/src/Functions/isIPAddressContainedIn.cpp @@ -4,291 +4,120 @@ #include #include #include +#include #include #include #include #include #include #include +#include -namespace DB + +#include +namespace DB::ErrorCodes { - namespace ErrorCodes - { - extern const int CANNOT_PARSE_TEXT; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - } + extern const int CANNOT_PARSE_TEXT; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int CANNOT_PARSE_NUMBER; } -namespace ipaddr +namespace { - class Address + +class IPAddressVariant +{ +public: + + explicit IPAddressVariant(const StringRef & address_str) { - public: - Address() = delete; + // IP address parser functions require that the input is + // NULL-terminated so we need to copy it. + const auto address_str_copy = std::string(address_str); - explicit Address(const StringRef & in) + UInt32 v4; + if (DB::parseIPv4(address_str_copy.c_str(), reinterpret_cast(&v4))) { - // IP address parser functions require that the input is - // NULL-terminated so we need to copy it. - const auto in_copy = std::string(in); - - UInt32 v4; - if (DB::parseIPv4(in_copy.c_str(), reinterpret_cast(&v4))) - { - addr = V4(v4); - } - else - { - V6 v6; - if (DB::parseIPv6(in_copy.c_str(), v6.addr.data())) - addr = std::move(v6); - else - throw DB::Exception("Neither IPv4 nor IPv6 address: " + in_copy, - DB::ErrorCodes::CANNOT_PARSE_TEXT); - } + addr = v4; } - - template - struct IPVersionBase + else { - IPVersionBase() - : addr {} {} - - explicit IPVersionBase(const std::array & octets) - : addr(octets) {} - - constexpr size_t numBits() const - { - return numOctets * 8; - } - - uint8_t operator[] (size_t i) const - { - assert(i >= 0 && i < numOctets); - return addr[i]; - } - - uint8_t & operator[] (size_t i) - { - assert(i >= 0 && i < numOctets); - return addr[i]; - } - - bool operator<= (const ConcreteType & rhs) const - { - for (size_t i = 0; i < numOctets; i++) - { - if ((*this)[i] < rhs[i]) return true; - if ((*this)[i] > rhs[i]) return false; - } - return true; - } - - bool operator>= (const ConcreteType & rhs) const - { - for (size_t i = 0; i < numOctets; i++) - { - if ((*this)[i] > rhs[i]) return true; - if ((*this)[i] < rhs[i]) return false; - } - return true; - } - - ConcreteType operator& (const ConcreteType & rhs) const - { - ConcreteType lhs(addr); - - for (size_t i = 0; i < numOctets; i++) - lhs[i] &= rhs[i]; - - return lhs; - } - - ConcreteType operator| (const ConcreteType & rhs) const - { - ConcreteType lhs(addr); - - for (size_t i = 0; i < numOctets; i++) - lhs[i] |= rhs[i]; - - return lhs; - } - - ConcreteType operator~ () const - { - ConcreteType tmp(addr); - - for (size_t i = 0; i < numOctets; i++) - tmp[i] = ~tmp[i]; - - return tmp; - } - - private: - // Big-endian - std::array addr; - friend class Address; - }; - - struct V4 : public IPVersionBase - { - V4() = default; - - explicit V4(UInt32 addr_) - { - addr[0] = (addr_ >> 24) & 0xFF; - addr[1] = (addr_ >> 16) & 0xFF; - addr[2] = (addr_ >> 8) & 0xFF; - addr[3] = addr_ & 0xFF; - } - - explicit V4(const std::array & components) - : IPVersionBase(components) {} - }; - - struct V6 : public IPVersionBase - { - V6() = default; - - explicit V6(const std::array & components) - : IPVersionBase(components) {} - }; - - constexpr const std::variant & variant() const - { - return addr; + addr = IPv6AddrType(); + bool success = DB::parseIPv6(address_str_copy.c_str(), std::get(addr).data()); + if (!success) + throw DB::Exception("Neither IPv4 nor IPv6 address: '" + address_str_copy + "'", + DB::ErrorCodes::CANNOT_PARSE_TEXT); } + } - private: - std::variant addr; - }; - - class CIDR + UInt32 asV4() const { - public: - CIDR() = delete; + if (const auto * val = std::get_if(&addr)) + return *val; + return 0; + } - explicit CIDR(const StringRef & in) - { - const auto in_view = std::string_view(in); - const auto pos_slash = in_view.find('/'); + const uint8_t * asV6() const + { + if (const auto * val = std::get_if(&addr)) + return val->data(); + return nullptr; + } - if (pos_slash == std::string_view::npos) - throw DB::Exception("The text does not contain '/': " + std::string(in_view), - DB::ErrorCodes::CANNOT_PARSE_TEXT); +private: + using IPv4AddrType = UInt32; + using IPv6AddrType = std::array; - prefix = Address(StringRef(in_view.substr(0, pos_slash))); + std::variant addr; +}; - // DB::parse() in ignores - // non-digit characters. std::stoi() skips whitespaces. We - // need to parse the prefix bits in a strict way. +struct IPAddressCIDR +{ + IPAddressVariant address; + UInt8 prefix; +}; - if (pos_slash + 1 == in_view.size()) - throw DB::Exception("The CIDR has no prefix bits: " + std::string(in_view), - DB::ErrorCodes::CANNOT_PARSE_TEXT); +IPAddressCIDR parseIPWithCIDR(const StringRef cidr_str) +{ + std::string_view cidr_str_view(cidr_str); + size_t pos_slash = cidr_str_view.find('/'); - bits = 0; - for (size_t i = pos_slash + 1; i < in_view.size(); i++) - { - const auto c = in_view[i]; - if (c >= '0' && c <= '9') - { - bits *= 10; - bits += c - '0'; - } - else - { - throw DB::Exception("The CIDR has a malformed prefix bits: " + std::string(in_view), - DB::ErrorCodes::CANNOT_PARSE_TEXT); - } - } + if (pos_slash == 0) + throw DB::Exception("Error parsing IP address with prefix: " + std::string(cidr_str), DB::ErrorCodes::CANNOT_PARSE_TEXT); + if (pos_slash == std::string_view::npos) + throw DB::Exception("The text does not contain '/': " + std::string(cidr_str), DB::ErrorCodes::CANNOT_PARSE_TEXT); - const size_t max_bits - = std::visit([&](const auto & addr_v) -> size_t - { - return addr_v.numBits(); - }, prefix->variant()); - if (bits > max_bits) - throw DB::Exception("The CIDR has an invalid prefix bits: " + std::string(in_view), - DB::ErrorCodes::CANNOT_PARSE_TEXT); - } + std::string_view addr_str = cidr_str_view.substr(0, pos_slash); + IPAddressVariant addr(StringRef{addr_str.data(), addr_str.size()}); - private: - template - static PrefixT toMask(uint8_t bits) - { - if constexpr (std::is_same_v) - { - return PrefixT(DB::getCIDRMaskIPv4(bits)); - } - else - { - return PrefixT(DB::getCIDRMaskIPv6(bits)); - } - } + UInt8 prefix = 0; + auto prefix_str = cidr_str_view.substr(pos_slash+1); - template - static PrefixT startOf(const PrefixT & prefix, uint8_t bits) - { - return prefix & toMask(bits); - } + const auto * prefix_str_end = prefix_str.data() + prefix_str.size(); + auto [parse_end, parse_error] = std::from_chars(prefix_str.data(), prefix_str_end, prefix); + UInt8 max_prefix = (addr.asV6() ? IPV6_BINARY_LENGTH : IPV4_BINARY_LENGTH) * 8; + bool has_error = parse_error != std::errc() || parse_end != prefix_str_end || prefix > max_prefix; + if (has_error) + throw DB::Exception("The CIDR has a malformed prefix bits: " + std::string(cidr_str), DB::ErrorCodes::CANNOT_PARSE_TEXT); - template - static PrefixT endOf(const PrefixT & prefix, uint8_t bits) - { - return prefix | ~toMask(bits); - } + return {addr, prefix}; +} - /* Convert a CIDR notation into an IP address range [start, end]. */ - template - static std::pair toRange(const PrefixT & prefix, uint8_t bits) - { - return std::make_pair(startOf(prefix, bits), endOf(prefix, bits)); - } +inline bool isAddressInRange(const IPAddressVariant & address, const IPAddressCIDR & cidr) +{ + if (const auto * cidr_v6 = cidr.address.asV6()) + { + if (const auto * addr_v6 = address.asV6()) + return DB::matchIPv6Subnet(addr_v6, cidr_v6, cidr.prefix); + } + else + { + if (!address.asV6()) + return DB::matchIPv4Subnet(address.asV4(), cidr.address.asV4(), cidr.prefix); + } + return false; +} - public: - bool contains(const Address & addr) const - { - return std::visit([&](const auto & addr_v) -> bool - { - return std::visit([&](const auto & prefix_v) -> bool - { - using AddrT = std::decay_t; - using PrefixT = std::decay_t; - - if constexpr (std::is_same_v) - { - if constexpr (std::is_same_v) - { - const auto range = toRange(prefix_v, bits); - return addr_v >= range.first && addr_v <= range.second; - } - else - { - return false; // IP version mismatch is not an error. - } - } - else - { - if constexpr (std::is_same_v) - { - const auto range = toRange(prefix_v, bits); - return addr_v >= range.first && addr_v <= range.second; - } - else - { - return false; // IP version mismatch is not an error. - } - } - }, prefix->variant()); - }, addr.variant()); - } - - private: - std::optional
prefix; // Guaranteed to have a value after construction. - uint8_t bits; - }; } namespace DB @@ -309,9 +138,8 @@ namespace DB if (const auto * col_addr_const = checkAndGetAnyColumnConst(col_addr)) { - // col_addr_const is constant and is either String or - // Nullable(String). We don't care which one it exactly is. - + // col_addr_const is constant and is either String or Nullable(String). + // We don't care which one it exactly is. if (const auto * col_cidr_const = checkAndGetAnyColumnConst(col_cidr)) return executeImpl(*col_addr_const, *col_cidr_const, return_type, input_rows_count); else @@ -359,13 +187,13 @@ namespace DB } else { - const auto addr = ipaddr::Address(col_addr.getDataAt(0)); - const auto cidr = ipaddr::CIDR(col_cidr.getDataAt(0)); + const auto addr = IPAddressVariant(col_addr.getDataAt(0)); + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(0)); ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(1); ColumnUInt8::Container & vec_res = col_res->getData(); - vec_res[0] = cidr.contains(addr) ? 1 : 0; + vec_res[0] = isAddressInRange(addr, cidr) ? 1 : 0; if (return_type->isNullable()) { @@ -396,7 +224,7 @@ namespace DB } else { - const auto addr = ipaddr::Address(col_addr.getDataAt(0)); + const auto addr = IPAddressVariant(col_addr.getDataAt (0)); ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); ColumnUInt8::Container & vec_res = col_res->getData(); @@ -414,8 +242,8 @@ namespace DB } else { - const auto cidr = ipaddr::CIDR(col_cidr.getDataAt(i)); - vec_res[i] = cidr.contains(addr) ? 1 : 0; + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); + vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; vec_null_map_res[i] = false; } } @@ -426,8 +254,8 @@ namespace DB { for (size_t i = 0; i < input_rows_count; i++) { - const auto cidr = ipaddr::CIDR(col_cidr.getDataAt(i)); - vec_res[i] = cidr.contains(addr) ? 1 : 0; + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); + vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; } return col_res; @@ -448,7 +276,7 @@ namespace DB } else { - const auto cidr = ipaddr::CIDR(col_cidr.getDataAt(0)); + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(0)); ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); ColumnUInt8::Container & vec_res = col_res->getData(); @@ -466,8 +294,8 @@ namespace DB } else { - const auto addr = ipaddr::Address(col_addr.getDataAt(i)); - vec_res[i] = cidr.contains(addr) ? 1 : 0; + const auto addr = IPAddressVariant(col_addr.getDataAt(i)); + vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; vec_null_map_res[i] = false; } } @@ -478,8 +306,8 @@ namespace DB { for (size_t i = 0; i < input_rows_count; i++) { - const auto addr = ipaddr::Address(col_addr.getDataAt(i)); - vec_res[i] = cidr.contains(addr) ? 1 : 0; + const auto addr = IPAddressVariant(col_addr.getDataAt(i)); + vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; } return col_res; @@ -506,10 +334,10 @@ namespace DB } else { - const auto addr = ipaddr::Address(col_addr.getDataAt(i)); - const auto cidr = ipaddr::CIDR(col_cidr.getDataAt(i)); + const auto addr = IPAddressVariant(col_addr.getDataAt(i)); + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); - vec_res[i] = cidr.contains(addr) ? 1 : 0; + vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; vec_null_map_res[i] = false; } } @@ -520,10 +348,10 @@ namespace DB { for (size_t i = 0; i < input_rows_count; i++) { - const auto addr = ipaddr::Address(col_addr.getDataAt(i)); - const auto cidr = ipaddr::CIDR(col_cidr.getDataAt(i)); + const auto addr = IPAddressVariant(col_addr.getDataAt(i)); + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); - vec_res[i] = cidr.contains(addr) ? 1 : 0; + vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; } return col_res; From 81ff4f4791d509e371460a8edd87e37c99f9c386 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Mar 2021 12:48:12 +0300 Subject: [PATCH 12/34] Remove Nummable support from isIPAddressContainedIn, do not use OverloadResolver --- src/Functions/isIPAddressContainedIn.cpp | 328 ++++-------------- .../01700_isIPAddressContainedIn.reference | 39 --- .../01700_isIPAddressContainedIn.sql | 47 +-- 3 files changed, 71 insertions(+), 343 deletions(-) diff --git a/src/Functions/isIPAddressContainedIn.cpp b/src/Functions/isIPAddressContainedIn.cpp index 45e2f15ed66..5f31011d49d 100644 --- a/src/Functions/isIPAddressContainedIn.cpp +++ b/src/Functions/isIPAddressContainedIn.cpp @@ -1,10 +1,8 @@ #include -#include #include #include #include #include -#include #include #include #include @@ -19,7 +17,7 @@ namespace DB::ErrorCodes { extern const int CANNOT_PARSE_TEXT; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int CANNOT_PARSE_NUMBER; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } namespace @@ -31,8 +29,8 @@ public: explicit IPAddressVariant(const StringRef & address_str) { - // IP address parser functions require that the input is - // NULL-terminated so we need to copy it. + /// IP address parser functions require that the input is + /// NULL-terminated so we need to copy it. const auto address_str_copy = std::string(address_str); UInt32 v4; @@ -122,49 +120,57 @@ inline bool isAddressInRange(const IPAddressVariant & address, const IPAddressCI namespace DB { - template - class ExecutableFunctionIsIPAddressContainedIn : public IExecutableFunctionImpl + class FunctionIsIPAddressContainedIn : public IFunction { public: - String getName() const override - { - return Name::name; - } + static constexpr auto name = "isIPAddressContainedIn"; + String getName() const override { return name; } + static FunctionPtr create(const Context &) { return std::make_shared(); } - ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type, size_t input_rows_count) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* return_type */, size_t input_rows_count) const override { const IColumn * col_addr = arguments[0].column.get(); const IColumn * col_cidr = arguments[1].column.get(); if (const auto * col_addr_const = checkAndGetAnyColumnConst(col_addr)) { - // col_addr_const is constant and is either String or Nullable(String). - // We don't care which one it exactly is. if (const auto * col_cidr_const = checkAndGetAnyColumnConst(col_cidr)) - return executeImpl(*col_addr_const, *col_cidr_const, return_type, input_rows_count); + return executeImpl(*col_addr_const, *col_cidr_const, input_rows_count); else - return executeImpl(*col_addr_const, *col_cidr, return_type, input_rows_count); + return executeImpl(*col_addr_const, *col_cidr, input_rows_count); } else { if (const auto * col_cidr_const = checkAndGetAnyColumnConst(col_cidr)) - return executeImpl(*col_addr, *col_cidr_const, return_type, input_rows_count); + return executeImpl(*col_addr, *col_cidr_const, input_rows_count); else return executeImpl(*col_addr, *col_cidr, input_rows_count); } } - bool useDefaultImplementationForNulls() const override + virtual DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - // We can't use the default implementation because that would end up - // parsing invalid addresses or prefixes at NULL fields, which would - // throw exceptions instead of returning NULL. - return false; + if (arguments.size() != 2) + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 2", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const DataTypePtr & addr_type = arguments[0]; + const DataTypePtr & prefix_type = arguments[1]; + + if (!isString(addr_type) || !isString(prefix_type)) + throw Exception("The arguments of function " + getName() + " must be String", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); } + virtual size_t getNumberOfArguments() const override { return 2; } + bool useDefaultImplementationForNulls() const override { return false; } + private: - // Like checkAndGetColumnConst() but this function doesn't - // care about the type of data column. + /// Like checkAndGetColumnConst() but this function doesn't + /// care about the type of data column. static const ColumnConst * checkAndGetAnyColumnConst(const IColumn * column) { if (!column || !isColumnConst(*column)) @@ -173,277 +179,81 @@ namespace DB return assert_cast(column); } - // Both columns are constant. - ColumnPtr executeImpl(const ColumnConst & col_addr_const, const ColumnConst & col_cidr_const, const DataTypePtr & return_type, size_t input_rows_count) const + /// Both columns are constant. + ColumnPtr executeImpl( + const ColumnConst & col_addr_const, + const ColumnConst & col_cidr_const, + size_t input_rows_count) const { const auto & col_addr = col_addr_const.getDataColumn(); const auto & col_cidr = col_cidr_const.getDataColumn(); - if (col_addr.isNullAt(0) || col_cidr.isNullAt(0)) - { - // If either of the arguments are NULL, the result is also NULL. - assert(return_type->isNullable()); - return return_type->createColumnConstWithDefaultValue(input_rows_count); - } - else - { - const auto addr = IPAddressVariant(col_addr.getDataAt(0)); - const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(0)); + const auto addr = IPAddressVariant(col_addr.getDataAt(0)); + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(0)); - ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(1); - ColumnUInt8::Container & vec_res = col_res->getData(); + ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(1); + ColumnUInt8::Container & vec_res = col_res->getData(); - vec_res[0] = isAddressInRange(addr, cidr) ? 1 : 0; + vec_res[0] = isAddressInRange(addr, cidr) ? 1 : 0; - if (return_type->isNullable()) - { - ColumnUInt8::MutablePtr col_null_map_res = ColumnUInt8::create(1); - ColumnUInt8::Container & vec_null_map_res = col_null_map_res->getData(); - - vec_null_map_res[0] = false; - - return ColumnConst::create(ColumnNullable::create(std::move(col_res), std::move(col_null_map_res)), input_rows_count); - } - else - { - return ColumnConst::create(std::move(col_res), input_rows_count); - } - } + return ColumnConst::create(std::move(col_res), input_rows_count); } - // Address is constant. - ColumnPtr executeImpl(const ColumnConst & col_addr_const, const IColumn & col_cidr, const DataTypePtr & return_type, size_t input_rows_count) const + /// Address is constant. + ColumnPtr executeImpl(const ColumnConst & col_addr_const, const IColumn & col_cidr, size_t input_rows_count) const { const auto & col_addr = col_addr_const.getDataColumn(); - if (col_addr.isNullAt(0)) + const auto addr = IPAddressVariant(col_addr.getDataAt (0)); + + ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); + ColumnUInt8::Container & vec_res = col_res->getData(); + + for (size_t i = 0; i < input_rows_count; i++) { - // It's constant NULL so the result is also constant NULL. - assert(return_type->isNullable()); - return return_type->createColumnConstWithDefaultValue(input_rows_count); - } - else - { - const auto addr = IPAddressVariant(col_addr.getDataAt (0)); - - ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); - ColumnUInt8::Container & vec_res = col_res->getData(); - - if (col_addr.isNullable() || col_cidr.isNullable()) - { - ColumnUInt8::MutablePtr col_null_map_res = ColumnUInt8::create(input_rows_count); - ColumnUInt8::Container & vec_null_map_res = col_null_map_res->getData(); - - for (size_t i = 0; i < input_rows_count; i++) - { - if (col_cidr.isNullAt(i)) - { - vec_null_map_res[i] = true; - } - else - { - const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); - vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; - vec_null_map_res[i] = false; - } - } - - return ColumnNullable::create(std::move(col_res), std::move(col_null_map_res)); - } - else - { - for (size_t i = 0; i < input_rows_count; i++) - { - const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); - vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; - } - - return col_res; - } + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); + vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; } + return col_res; } - // CIDR is constant. - ColumnPtr executeImpl(const IColumn & col_addr, const ColumnConst & col_cidr_const, const DataTypePtr & return_type, size_t input_rows_count) const + /// CIDR is constant. + ColumnPtr executeImpl(const IColumn & col_addr, const ColumnConst & col_cidr_const, size_t input_rows_count) const { const auto & col_cidr = col_cidr_const.getDataColumn(); - if (col_cidr.isNullAt(0)) + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(0)); + + ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); + ColumnUInt8::Container & vec_res = col_res->getData(); + for (size_t i = 0; i < input_rows_count; i++) { - // It's constant NULL so the result is also constant NULL. - assert(return_type->isNullable()); - return return_type->createColumnConstWithDefaultValue(input_rows_count); - } - else - { - const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(0)); - - ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); - ColumnUInt8::Container & vec_res = col_res->getData(); - - if (col_addr.isNullable() || col_cidr.isNullable()) - { - ColumnUInt8::MutablePtr col_null_map_res = ColumnUInt8::create(input_rows_count); - ColumnUInt8::Container & vec_null_map_res = col_null_map_res->getData(); - - for (size_t i = 0; i < input_rows_count; i++) - { - if (col_addr.isNullAt(i)) - { - vec_null_map_res[i] = true; - } - else - { - const auto addr = IPAddressVariant(col_addr.getDataAt(i)); - vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; - vec_null_map_res[i] = false; - } - } - - return ColumnNullable::create(std::move(col_res), std::move(col_null_map_res)); - } - else - { - for (size_t i = 0; i < input_rows_count; i++) - { - const auto addr = IPAddressVariant(col_addr.getDataAt(i)); - vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; - } - - return col_res; - } + const auto addr = IPAddressVariant(col_addr.getDataAt(i)); + vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; } + return col_res; } - // Neither are constant. + /// Neither are constant. ColumnPtr executeImpl(const IColumn & col_addr, const IColumn & col_cidr, size_t input_rows_count) const { ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); ColumnUInt8::Container & vec_res = col_res->getData(); - if (col_addr.isNullable() || col_cidr.isNullable()) + for (size_t i = 0; i < input_rows_count; i++) { - ColumnUInt8::MutablePtr col_null_map_res = ColumnUInt8::create(input_rows_count); - ColumnUInt8::Container & vec_null_map_res = col_null_map_res->getData(); + const auto addr = IPAddressVariant(col_addr.getDataAt(i)); + const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); - for (size_t i = 0; i < input_rows_count; i++) - { - if (col_addr.isNullAt(i) || col_cidr.isNullAt(i)) - { - vec_null_map_res[i] = true; - } - else - { - const auto addr = IPAddressVariant(col_addr.getDataAt(i)); - const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); - - vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; - vec_null_map_res[i] = false; - } - } - - return ColumnNullable::create(std::move(col_res), std::move(col_null_map_res)); + vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; } - else - { - for (size_t i = 0; i < input_rows_count; i++) - { - const auto addr = IPAddressVariant(col_addr.getDataAt(i)); - const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); - vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; - } - - return col_res; - } + return col_res; } }; - template - class FunctionBaseIsIPAddressContainedIn : public IFunctionBaseImpl - { - public: - explicit FunctionBaseIsIPAddressContainedIn(DataTypes argument_types_, DataTypePtr return_type_) - : argument_types(std::move(argument_types_)) - , return_type(std::move(return_type_)) {} - - String getName() const override - { - return Name::name; - } - - const DataTypes & getArgumentTypes() const override - { - return argument_types; - } - - const DataTypePtr & getResultType() const override - { - return return_type; - } - - ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName &) const override - { - return std::make_unique>(); - } - - private: - DataTypes argument_types; - DataTypePtr return_type; - }; - - template - class IsIPAddressContainedInOverloadResolver : public IFunctionOverloadResolverImpl - { - public: - static constexpr auto name = Name::name; - - static FunctionOverloadResolverImplPtr create(const Context &) - { - return std::make_unique>(); - } - - String getName() const override - { - return Name::name; - } - - FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override - { - const DataTypePtr & addr_type = removeNullable(arguments[0].type); - const DataTypePtr & prefix_type = removeNullable(arguments[1].type); - DataTypes argument_types = { addr_type, prefix_type }; - - /* The arguments can be any of Nullable(NULL), Nullable(String), and - * String. We can't do this check in getReturnType() because it - * won't be called when there are any constant NULLs in the - * arguments. */ - if (!(WhichDataType(addr_type).isNothing() || isString(addr_type)) || - !(WhichDataType(prefix_type).isNothing() || isString(prefix_type))) - throw Exception("The arguments of function " + getName() + " must be String", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_unique>(argument_types, return_type); - } - - DataTypePtr getReturnType(const DataTypes &) const override - { - return std::make_shared(); - } - - size_t getNumberOfArguments() const override - { - return 2; - } - }; - - struct NameIsIPAddressContainedIn - { - static constexpr auto name = "isIPAddressContainedIn"; - }; - void registerFunctionIsIPAddressContainedIn(FunctionFactory & factory) { - factory.registerFunction>(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/01700_isIPAddressContainedIn.reference b/tests/queries/0_stateless/01700_isIPAddressContainedIn.reference index 98650dd69f4..578af374123 100644 --- a/tests/queries/0_stateless/01700_isIPAddressContainedIn.reference +++ b/tests/queries/0_stateless/01700_isIPAddressContainedIn.reference @@ -33,44 +33,5 @@ 0 0 0 -# Arguments can be nullable. -## Nullable address -\N -\N -1 -## Nullable prefix -\N -\N -1 -## Both nullable -\N -\N -1 -# Non-constant nullable arguments -## Non-constant address -127.0.0.1 127.0.0.0/8 1 -\N 127.0.0.0/8 \N -127.0.0.1 \N \N -\N \N \N -127.0.0.1 \N \N -\N \N \N -127.0.0.1 127.0.0.0/8 1 -\N 127.0.0.0/8 \N -127.0.0.1 127.0.0.0/8 1 -## Non-constant prefix -127.0.0.1 127.0.0.0/8 1 -127.0.0.1 \N \N -\N 127.0.0.0/8 \N -\N \N \N -\N 127.0.0.0/8 \N -\N \N \N -127.0.0.1 127.0.0.0/8 1 -127.0.0.1 \N \N -127.0.0.1 127.0.0.0/8 1 -## Both non-constant -127.0.0.1 127.0.0.0/8 1 -127.0.0.1 \N \N -\N 127.0.0.0/8 \N -\N \N \N # Unparsable arguments # Wrong argument types diff --git a/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql b/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql index 0618aa0cfa5..7c76d93443f 100644 --- a/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql +++ b/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql @@ -1,4 +1,3 @@ --- SELECT '# Invocation with constants'; SELECT isIPAddressContainedIn('127.0.0.1', '127.0.0.0/8'); @@ -7,25 +6,21 @@ SELECT isIPAddressContainedIn('128.0.0.1', '127.0.0.0/8'); SELECT isIPAddressContainedIn('ffff::1', 'ffff::/16'); SELECT isIPAddressContainedIn('fffe::1', 'ffff::/16'); --- SELECT '# Invocation with non-constant addresses'; WITH arrayJoin(['192.168.99.255', '192.168.100.1', '192.168.103.255', '192.168.104.0']) as addr, '192.168.100.0/22' as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); WITH arrayJoin(['::192.168.99.255', '::192.168.100.1', '::192.168.103.255', '::192.168.104.0']) as addr, '::192.168.100.0/118' as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); --- SELECT '# Invocation with non-constant prefixes'; WITH '192.168.100.1' as addr, arrayJoin(['192.168.100.0/22', '192.168.100.0/24', '192.168.100.0/32']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); WITH '::192.168.100.1' as addr, arrayJoin(['::192.168.100.0/118', '::192.168.100.0/120', '::192.168.100.0/128']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); --- SELECT '# Invocation with non-constants'; WITH arrayJoin(['192.168.100.1', '192.168.103.255']) as addr, arrayJoin(['192.168.100.0/22', '192.168.100.0/24']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); WITH arrayJoin(['::192.168.100.1', '::192.168.103.255']) as addr, arrayJoin(['::192.168.100.0/118', '::192.168.100.0/120']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); --- SELECT '# Mismatching IP versions is not an error.'; SELECT isIPAddressContainedIn('127.0.0.1', 'ffff::/16'); @@ -33,54 +28,16 @@ SELECT isIPAddressContainedIn('127.0.0.1', '::127.0.0.1/128'); SELECT isIPAddressContainedIn('::1', '127.0.0.0/8'); SELECT isIPAddressContainedIn('::127.0.0.1', '127.0.0.1/32'); --- -SELECT '# Arguments can be nullable.'; - -SELECT '## Nullable address'; -SELECT isIPAddressContainedIn(NULL , '127.0.0.0/8'); -SELECT isIPAddressContainedIn(CAST(NULL, 'Nullable(String)') , '127.0.0.0/8'); -SELECT isIPAddressContainedIn(CAST('127.0.0.1', 'Nullable(String)'), '127.0.0.0/8'); - -SELECT '## Nullable prefix'; -SELECT isIPAddressContainedIn('127.0.0.1', NULL); -SELECT isIPAddressContainedIn('127.0.0.1', CAST(NULL, 'Nullable(String)')); -SELECT isIPAddressContainedIn('127.0.0.1', CAST('127.0.0.0/8', 'Nullable(String)')); - -SELECT '## Both nullable'; -SELECT isIPAddressContainedIn(NULL , NULL); -SELECT isIPAddressContainedIn(CAST(NULL, 'Nullable(String)') , CAST(NULL, 'Nullable(String)')); -SELECT isIPAddressContainedIn(CAST('127.0.0.1', 'Nullable(String)'), CAST('127.0.0.0/8', 'Nullable(String)')); - --- -SELECT '# Non-constant nullable arguments'; - -SELECT '## Non-constant address'; -WITH arrayJoin(['127.0.0.1', NULL]) as addr, '127.0.0.0/8' as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); -WITH arrayJoin(['127.0.0.1', NULL]) as addr, NULL as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); -WITH arrayJoin(['127.0.0.1', NULL]) as addr, CAST(NULL, 'Nullable(String)') as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); -WITH arrayJoin(['127.0.0.1', NULL]) as addr, CAST('127.0.0.0/8', 'Nullable(String)') as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); -WITH arrayJoin(['127.0.0.1']) as addr, CAST('127.0.0.0/8', 'Nullable(String)') as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); - -SELECT '## Non-constant prefix'; -WITH '127.0.0.1' as addr, arrayJoin(['127.0.0.0/8', NULL]) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); -WITH NULL as addr, arrayJoin(['127.0.0.0/8', NULL]) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); -WITH CAST(NULL, 'Nullable(String)') as addr, arrayJoin(['127.0.0.0/8', NULL]) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); -WITH CAST('127.0.0.1', 'Nullable(String)') as addr, arrayJoin(['127.0.0.0/8', NULL]) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); -WITH CAST('127.0.0.1', 'Nullable(String)') as addr, arrayJoin(['127.0.0.0/8']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); - -SELECT '## Both non-constant'; -WITH arrayJoin(['127.0.0.1', NULL]) as addr, arrayJoin(['127.0.0.0/8', NULL]) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); - --- SELECT '# Unparsable arguments'; SELECT isIPAddressContainedIn('unparsable', '127.0.0.0/8'); -- { serverError 6 } SELECT isIPAddressContainedIn('127.0.0.1', 'unparsable'); -- { serverError 6 } --- SELECT '# Wrong argument types'; SELECT isIPAddressContainedIn(100, '127.0.0.0/8'); -- { serverError 43 } +SELECT isIPAddressContainedIn(NULL, '127.0.0.0/8'); -- { serverError 43 } +SELECT isIPAddressContainedIn(CAST(NULL, 'Nullable(String)'), '127.0.0.0/8'); -- { serverError 43 } SELECT isIPAddressContainedIn('127.0.0.1', 100); -- { serverError 43 } SELECT isIPAddressContainedIn(100, NULL); -- { serverError 43 } WITH arrayJoin([NULL, NULL, NULL, NULL]) AS prefix SELECT isIPAddressContainedIn([NULL, NULL, 0, 255, 0], prefix); -- { serverError 43 } From ae4b8e3181a1b00b3e15f7a4560f258c4a2d8013 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Mar 2021 12:56:11 +0300 Subject: [PATCH 13/34] Rename isIPAddressContainedIn -> isIPAddressInRange --- src/Functions/isIPAddressContainedIn.cpp | 2 +- .../01700_isIPAddressContainedIn.sql | 44 +++++++++---------- 2 files changed, 23 insertions(+), 23 deletions(-) diff --git a/src/Functions/isIPAddressContainedIn.cpp b/src/Functions/isIPAddressContainedIn.cpp index 5f31011d49d..72ec9e21f50 100644 --- a/src/Functions/isIPAddressContainedIn.cpp +++ b/src/Functions/isIPAddressContainedIn.cpp @@ -123,7 +123,7 @@ namespace DB class FunctionIsIPAddressContainedIn : public IFunction { public: - static constexpr auto name = "isIPAddressContainedIn"; + static constexpr auto name = "isIPAddressInRange"; String getName() const override { return name; } static FunctionPtr create(const Context &) { return std::make_shared(); } diff --git a/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql b/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql index 7c76d93443f..2fdff52418a 100644 --- a/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql +++ b/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql @@ -1,43 +1,43 @@ SELECT '# Invocation with constants'; -SELECT isIPAddressContainedIn('127.0.0.1', '127.0.0.0/8'); -SELECT isIPAddressContainedIn('128.0.0.1', '127.0.0.0/8'); +SELECT isIPAddressInRange('127.0.0.1', '127.0.0.0/8'); +SELECT isIPAddressInRange('128.0.0.1', '127.0.0.0/8'); -SELECT isIPAddressContainedIn('ffff::1', 'ffff::/16'); -SELECT isIPAddressContainedIn('fffe::1', 'ffff::/16'); +SELECT isIPAddressInRange('ffff::1', 'ffff::/16'); +SELECT isIPAddressInRange('fffe::1', 'ffff::/16'); SELECT '# Invocation with non-constant addresses'; -WITH arrayJoin(['192.168.99.255', '192.168.100.1', '192.168.103.255', '192.168.104.0']) as addr, '192.168.100.0/22' as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); -WITH arrayJoin(['::192.168.99.255', '::192.168.100.1', '::192.168.103.255', '::192.168.104.0']) as addr, '::192.168.100.0/118' as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH arrayJoin(['192.168.99.255', '192.168.100.1', '192.168.103.255', '192.168.104.0']) as addr, '192.168.100.0/22' as prefix SELECT addr, prefix, isIPAddressInRange(addr, prefix); +WITH arrayJoin(['::192.168.99.255', '::192.168.100.1', '::192.168.103.255', '::192.168.104.0']) as addr, '::192.168.100.0/118' as prefix SELECT addr, prefix, isIPAddressInRange(addr, prefix); SELECT '# Invocation with non-constant prefixes'; -WITH '192.168.100.1' as addr, arrayJoin(['192.168.100.0/22', '192.168.100.0/24', '192.168.100.0/32']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); -WITH '::192.168.100.1' as addr, arrayJoin(['::192.168.100.0/118', '::192.168.100.0/120', '::192.168.100.0/128']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH '192.168.100.1' as addr, arrayJoin(['192.168.100.0/22', '192.168.100.0/24', '192.168.100.0/32']) as prefix SELECT addr, prefix, isIPAddressInRange(addr, prefix); +WITH '::192.168.100.1' as addr, arrayJoin(['::192.168.100.0/118', '::192.168.100.0/120', '::192.168.100.0/128']) as prefix SELECT addr, prefix, isIPAddressInRange(addr, prefix); SELECT '# Invocation with non-constants'; -WITH arrayJoin(['192.168.100.1', '192.168.103.255']) as addr, arrayJoin(['192.168.100.0/22', '192.168.100.0/24']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); -WITH arrayJoin(['::192.168.100.1', '::192.168.103.255']) as addr, arrayJoin(['::192.168.100.0/118', '::192.168.100.0/120']) as prefix SELECT addr, prefix, isIPAddressContainedIn(addr, prefix); +WITH arrayJoin(['192.168.100.1', '192.168.103.255']) as addr, arrayJoin(['192.168.100.0/22', '192.168.100.0/24']) as prefix SELECT addr, prefix, isIPAddressInRange(addr, prefix); +WITH arrayJoin(['::192.168.100.1', '::192.168.103.255']) as addr, arrayJoin(['::192.168.100.0/118', '::192.168.100.0/120']) as prefix SELECT addr, prefix, isIPAddressInRange(addr, prefix); SELECT '# Mismatching IP versions is not an error.'; -SELECT isIPAddressContainedIn('127.0.0.1', 'ffff::/16'); -SELECT isIPAddressContainedIn('127.0.0.1', '::127.0.0.1/128'); -SELECT isIPAddressContainedIn('::1', '127.0.0.0/8'); -SELECT isIPAddressContainedIn('::127.0.0.1', '127.0.0.1/32'); +SELECT isIPAddressInRange('127.0.0.1', 'ffff::/16'); +SELECT isIPAddressInRange('127.0.0.1', '::127.0.0.1/128'); +SELECT isIPAddressInRange('::1', '127.0.0.0/8'); +SELECT isIPAddressInRange('::127.0.0.1', '127.0.0.1/32'); SELECT '# Unparsable arguments'; -SELECT isIPAddressContainedIn('unparsable', '127.0.0.0/8'); -- { serverError 6 } -SELECT isIPAddressContainedIn('127.0.0.1', 'unparsable'); -- { serverError 6 } +SELECT isIPAddressInRange('unparsable', '127.0.0.0/8'); -- { serverError 6 } +SELECT isIPAddressInRange('127.0.0.1', 'unparsable'); -- { serverError 6 } SELECT '# Wrong argument types'; -SELECT isIPAddressContainedIn(100, '127.0.0.0/8'); -- { serverError 43 } -SELECT isIPAddressContainedIn(NULL, '127.0.0.0/8'); -- { serverError 43 } -SELECT isIPAddressContainedIn(CAST(NULL, 'Nullable(String)'), '127.0.0.0/8'); -- { serverError 43 } -SELECT isIPAddressContainedIn('127.0.0.1', 100); -- { serverError 43 } -SELECT isIPAddressContainedIn(100, NULL); -- { serverError 43 } -WITH arrayJoin([NULL, NULL, NULL, NULL]) AS prefix SELECT isIPAddressContainedIn([NULL, NULL, 0, 255, 0], prefix); -- { serverError 43 } +SELECT isIPAddressInRange(100, '127.0.0.0/8'); -- { serverError 43 } +SELECT isIPAddressInRange(NULL, '127.0.0.0/8'); -- { serverError 43 } +SELECT isIPAddressInRange(CAST(NULL, 'Nullable(String)'), '127.0.0.0/8'); -- { serverError 43 } +SELECT isIPAddressInRange('127.0.0.1', 100); -- { serverError 43 } +SELECT isIPAddressInRange(100, NULL); -- { serverError 43 } +WITH arrayJoin([NULL, NULL, NULL, NULL]) AS prefix SELECT isIPAddressInRange([NULL, NULL, 0, 255, 0], prefix); -- { serverError 43 } From b4c2980bf5efde97f1c43762811daf2a18aef458 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Mar 2021 13:24:52 +0300 Subject: [PATCH 14/34] Add test cases for isIPAddressInRange --- .../01700_isIPAddressContainedIn.reference | 9 ++++++++ .../01700_isIPAddressContainedIn.sql | 21 +++++++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/tests/queries/0_stateless/01700_isIPAddressContainedIn.reference b/tests/queries/0_stateless/01700_isIPAddressContainedIn.reference index 578af374123..b2f282d2183 100644 --- a/tests/queries/0_stateless/01700_isIPAddressContainedIn.reference +++ b/tests/queries/0_stateless/01700_isIPAddressContainedIn.reference @@ -28,6 +28,15 @@ ::192.168.100.1 ::192.168.100.0/120 1 ::192.168.103.255 ::192.168.100.0/118 1 ::192.168.103.255 ::192.168.100.0/120 0 +# Check with dense table +1 +1 +1 +1 +1 +1 +1 +1 # Mismatching IP versions is not an error. 0 0 diff --git a/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql b/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql index 2fdff52418a..29c2bcb220d 100644 --- a/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql +++ b/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql @@ -21,6 +21,27 @@ SELECT '# Invocation with non-constants'; WITH arrayJoin(['192.168.100.1', '192.168.103.255']) as addr, arrayJoin(['192.168.100.0/22', '192.168.100.0/24']) as prefix SELECT addr, prefix, isIPAddressInRange(addr, prefix); WITH arrayJoin(['::192.168.100.1', '::192.168.103.255']) as addr, arrayJoin(['::192.168.100.0/118', '::192.168.100.0/120']) as prefix SELECT addr, prefix, isIPAddressInRange(addr, prefix); +SELECT '# Check with dense table'; + +DROP TABLE IF EXISTS test_data; +CREATE TABLE test_data (cidr String) ENGINE = Memory; + +INSERT INTO test_data +SELECT + IPv4NumToString(IPv4CIDRToRange(IPv4StringToNum('255.255.255.255'), toUInt8(number)).1) || '/' || toString(number) AS cidr +FROM system.numbers LIMIT 33; + +SELECT sum(isIPAddressInRange('0.0.0.0', cidr)) == 1 FROM test_data; +SELECT sum(isIPAddressInRange('127.0.0.0', cidr)) == 1 FROM test_data; +SELECT sum(isIPAddressInRange('128.0.0.0', cidr)) == 2 FROM test_data; +SELECT sum(isIPAddressInRange('255.0.0.0', cidr)) == 9 FROM test_data; +SELECT sum(isIPAddressInRange('255.0.0.1', cidr)) == 9 FROM test_data; +SELECT sum(isIPAddressInRange('255.0.0.255', cidr)) == 9 FROM test_data; +SELECT sum(isIPAddressInRange('255.255.255.255', cidr)) == 33 FROM test_data; +SELECT sum(isIPAddressInRange('255.255.255.254', cidr)) == 32 FROM test_data; + +DROP TABLE IF EXISTS test_data; + SELECT '# Mismatching IP versions is not an error.'; SELECT isIPAddressInRange('127.0.0.1', 'ffff::/16'); From 7969bb0fe72e77e9d96104e893dee66136f7565b Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Mar 2021 13:27:27 +0300 Subject: [PATCH 15/34] Rename test file isIPAddressContainedIn to ip_address_in_range --- ...sContainedIn.reference => 01774_ip_address_in_range.reference} | 0 ...0_isIPAddressContainedIn.sql => 01774_ip_address_in_range.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01700_isIPAddressContainedIn.reference => 01774_ip_address_in_range.reference} (100%) rename tests/queries/0_stateless/{01700_isIPAddressContainedIn.sql => 01774_ip_address_in_range.sql} (100%) diff --git a/tests/queries/0_stateless/01700_isIPAddressContainedIn.reference b/tests/queries/0_stateless/01774_ip_address_in_range.reference similarity index 100% rename from tests/queries/0_stateless/01700_isIPAddressContainedIn.reference rename to tests/queries/0_stateless/01774_ip_address_in_range.reference diff --git a/tests/queries/0_stateless/01700_isIPAddressContainedIn.sql b/tests/queries/0_stateless/01774_ip_address_in_range.sql similarity index 100% rename from tests/queries/0_stateless/01700_isIPAddressContainedIn.sql rename to tests/queries/0_stateless/01774_ip_address_in_range.sql From ada4a2ba00de21d5bbf152b9dad919c09fce0d2c Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Mar 2021 13:43:21 +0300 Subject: [PATCH 16/34] Upd doc for isIPAddressInRange --- .../functions/ip-address-functions.md | 24 ++++----- .../functions/ip-address-functions.md | 51 +++++++++++++++++++ 2 files changed, 63 insertions(+), 12 deletions(-) diff --git a/docs/en/sql-reference/functions/ip-address-functions.md b/docs/en/sql-reference/functions/ip-address-functions.md index b293cd2d4a7..0b5dd7160b8 100644 --- a/docs/en/sql-reference/functions/ip-address-functions.md +++ b/docs/en/sql-reference/functions/ip-address-functions.md @@ -394,19 +394,19 @@ Result: └──────────────────┴────────────────────┘ ``` -## isIPAddressContainedIn {#isipaddresscontainedin} +## isIPAddressInRange {#isipaddressinrange} Determines if an IP address is contained in a network represented in the [CIDR](https://en.wikipedia.org/wiki/Classless_Inter-Domain_Routing) notation. Returns `1` if true, or `0` otherwise. **Syntax** ``` sql -isIPAddressContainedIn(address, prefix) +isIPAddressInRange(address, prefix) ``` -This function accepts both IPv4 and IPv6 addresses (and networks) represented as strings. It returns `0` if the IP version of the address and the CIDR don't match. It returns `NULL` if either of the arguments is `NULL`. +This function accepts both IPv4 and IPv6 addresses (and networks) represented as strings. It returns `0` if the IP version of the address and the CIDR don't match. -**Parameters** +**Arguments** - `address` — An IPv4 or IPv6 address. [String](../../sql-reference/data-types/string.md). - `prefix` — An IPv4 or IPv6 network prefix in CIDR. [String](../../sql-reference/data-types/string.md). @@ -422,27 +422,27 @@ Type: [UInt8](../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT isIPAddressContainedIn('127.0.0.1', '127.0.0.0/8') +SELECT isIPAddressInRange('127.0.0.1', '127.0.0.0/8') ``` Result: ``` text -┌─isIPAddressContainedIn('127.0.0.1', '127.0.0.0/8')─┐ -│ 1 │ -└────────────────────────────────────────────────────┘ +┌─isIPAddressInRange('127.0.0.1', '127.0.0.0/8')─┐ +│ 1 │ +└────────────────────────────────────────────────┘ ``` Query: ``` sql -SELECT isIPAddressContainedIn('127.0.0.1', 'ffff::/16') +SELECT isIPAddressInRange('127.0.0.1', 'ffff::/16') ``` Result: ``` text -┌─isIPAddressContainedIn('127.0.0.1', 'ffff::/16')─┐ -│ 0 │ -└──────────────────────────────────────────────────┘ +┌─isIPAddressInRange('127.0.0.1', 'ffff::/16')─┐ +│ 0 │ +└──────────────────────────────────────────────┘ ``` diff --git a/docs/ru/sql-reference/functions/ip-address-functions.md b/docs/ru/sql-reference/functions/ip-address-functions.md index 96adad10621..d7f6d2f7618 100644 --- a/docs/ru/sql-reference/functions/ip-address-functions.md +++ b/docs/ru/sql-reference/functions/ip-address-functions.md @@ -395,3 +395,54 @@ SELECT addr, isIPv6String(addr) FROM ( SELECT ['::', '1111::ffff', '::ffff:127.0 └──────────────────┴────────────────────┘ ``` +## isIPAddressInRange {#isipaddressinrange} + +Проверяет попадает ли IP адрес в интервал, заданный в [CIDR](https://en.wikipedia.org/wiki/Classless_Inter-Domain_Routing) нотации. + +**Syntax** + +``` sql +isIPAddressInRange(address, prefix) +``` +Функция принимает IPv4 или IPv6 адрес виде строки. Возвращает `0`, если версия адреса и интервала не совпадают. + +**Аргументы** + +- `address` — IPv4 или IPv6 адрес. [String](../../sql-reference/data-types/string.md). +- `prefix` — IPv4 или IPv6 подсеть, заданная в CIDR нотации. [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +- `1` или `0`. + +Тип: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Примеры** + +Запрос: + +``` sql +SELECT isIPAddressInRange('127.0.0.1', '127.0.0.0/8') +``` + +Результат: + +``` text +┌─isIPAddressInRange('127.0.0.1', '127.0.0.0/8')─┐ +│ 1 │ +└────────────────────────────────────────────────┘ +``` + +Запрос: + +``` sql +SELECT isIPAddressInRange('127.0.0.1', 'ffff::/16') +``` + +Результат: + +``` text +┌─isIPAddressInRange('127.0.0.1', 'ffff::/16')─┐ +│ 0 │ +└──────────────────────────────────────────────┘ +``` From 2c06eb8382633ed1bd25bae63cf81ca6e469ac0f Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 29 Mar 2021 14:59:28 +0300 Subject: [PATCH 17/34] Relax covering part check on mutation --- src/Storages/StorageReplicatedMergeTree.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0849f65477d..195730b3548 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1663,9 +1663,10 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM if (source_part->name != source_part_name) { - throw Exception("Part " + source_part_name + " is covered by " + source_part->name - + " but should be mutated to " + entry.new_part_name + ". This is a bug.", - ErrorCodes::LOGICAL_ERROR); + LOG_WARNING("Part " + source_part_name + " is covered by " + source_part->name + + " but should be mutated to " + entry.new_part_name + ". " + + "Possibly the mutation of this part is not needed and will be skipped. This shouldn't happen often."); + return false; } /// TODO - some better heuristic? From f439e17ece5ef3e24eabfcf927cddf51748bd36f Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 29 Mar 2021 15:41:26 +0300 Subject: [PATCH 18/34] Fix build --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 195730b3548..0fb9863c827 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1663,7 +1663,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM if (source_part->name != source_part_name) { - LOG_WARNING("Part " + source_part_name + " is covered by " + source_part->name + LOG_WARNING(log, "Part " + source_part_name + " is covered by " + source_part->name + " but should be mutated to " + entry.new_part_name + ". " + "Possibly the mutation of this part is not needed and will be skipped. This shouldn't happen often."); return false; From 02a0caf28bdc3b9a5e34b613482c290d493f73f6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Mar 2021 15:50:34 +0300 Subject: [PATCH 19/34] Fix build isIPAddressContainedIn --- src/Common/IPv6ToBinary.cpp | 2 +- src/Functions/isIPAddressContainedIn.cpp | 16 ++++++++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Common/IPv6ToBinary.cpp b/src/Common/IPv6ToBinary.cpp index a5ceaa72022..a8363a46de7 100644 --- a/src/Common/IPv6ToBinary.cpp +++ b/src/Common/IPv6ToBinary.cpp @@ -110,7 +110,7 @@ bool matchIPv6Subnet(const uint8_t * addr, const uint8_t * cidr_addr, UInt8 pref size_t i = 0; for (; prefix >= 8; ++i, prefix -= 8) { - if (target[i] != cidr_addr[i]) + if (addr[i] != cidr_addr[i]) return false; } if (prefix == 0) diff --git a/src/Functions/isIPAddressContainedIn.cpp b/src/Functions/isIPAddressContainedIn.cpp index 72ec9e21f50..d31e03cb63e 100644 --- a/src/Functions/isIPAddressContainedIn.cpp +++ b/src/Functions/isIPAddressContainedIn.cpp @@ -88,17 +88,17 @@ IPAddressCIDR parseIPWithCIDR(const StringRef cidr_str) std::string_view addr_str = cidr_str_view.substr(0, pos_slash); IPAddressVariant addr(StringRef{addr_str.data(), addr_str.size()}); - UInt8 prefix = 0; + uint8_t prefix = 0; auto prefix_str = cidr_str_view.substr(pos_slash+1); const auto * prefix_str_end = prefix_str.data() + prefix_str.size(); auto [parse_end, parse_error] = std::from_chars(prefix_str.data(), prefix_str_end, prefix); - UInt8 max_prefix = (addr.asV6() ? IPV6_BINARY_LENGTH : IPV4_BINARY_LENGTH) * 8; + uint8_t max_prefix = (addr.asV6() ? IPV6_BINARY_LENGTH : IPV4_BINARY_LENGTH) * 8; bool has_error = parse_error != std::errc() || parse_end != prefix_str_end || prefix > max_prefix; if (has_error) throw DB::Exception("The CIDR has a malformed prefix bits: " + std::string(cidr_str), DB::ErrorCodes::CANNOT_PARSE_TEXT); - return {addr, prefix}; + return {addr, static_cast(prefix)}; } inline bool isAddressInRange(const IPAddressVariant & address, const IPAddressCIDR & cidr) @@ -180,10 +180,10 @@ namespace DB } /// Both columns are constant. - ColumnPtr executeImpl( + static ColumnPtr executeImpl( const ColumnConst & col_addr_const, const ColumnConst & col_cidr_const, - size_t input_rows_count) const + size_t input_rows_count) { const auto & col_addr = col_addr_const.getDataColumn(); const auto & col_cidr = col_cidr_const.getDataColumn(); @@ -200,7 +200,7 @@ namespace DB } /// Address is constant. - ColumnPtr executeImpl(const ColumnConst & col_addr_const, const IColumn & col_cidr, size_t input_rows_count) const + static ColumnPtr executeImpl(const ColumnConst & col_addr_const, const IColumn & col_cidr, size_t input_rows_count) { const auto & col_addr = col_addr_const.getDataColumn(); @@ -218,7 +218,7 @@ namespace DB } /// CIDR is constant. - ColumnPtr executeImpl(const IColumn & col_addr, const ColumnConst & col_cidr_const, size_t input_rows_count) const + static ColumnPtr executeImpl(const IColumn & col_addr, const ColumnConst & col_cidr_const, size_t input_rows_count) { const auto & col_cidr = col_cidr_const.getDataColumn(); @@ -235,7 +235,7 @@ namespace DB } /// Neither are constant. - ColumnPtr executeImpl(const IColumn & col_addr, const IColumn & col_cidr, size_t input_rows_count) const + static ColumnPtr executeImpl(const IColumn & col_addr, const IColumn & col_cidr, size_t input_rows_count) { ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); ColumnUInt8::Container & vec_res = col_res->getData(); From 0876b146a52b18a1035e4c0154147fb3497699b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Mar 2021 19:54:07 +0300 Subject: [PATCH 20/34] More coarse test for DateLUT --- src/Common/tests/gtest_DateLUTImpl.cpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Common/tests/gtest_DateLUTImpl.cpp b/src/Common/tests/gtest_DateLUTImpl.cpp index 4be04e44fc1..74fd7cb6149 100644 --- a/src/Common/tests/gtest_DateLUTImpl.cpp +++ b/src/Common/tests/gtest_DateLUTImpl.cpp @@ -470,8 +470,8 @@ INSTANTIATE_TEST_SUITE_P(AllTimezones_Year2010, ::testing::ValuesIn(allTimezones()), ::testing::ValuesIn(std::initializer_list{ // Values from tests/date_lut3.cpp - {YYYYMMDDToDay(20101031), YYYYMMDDToDay(20101101), 15 * 60}, - {YYYYMMDDToDay(20100328), YYYYMMDDToDay(20100330), 15 * 60} + {YYYYMMDDToDay(20101031), YYYYMMDDToDay(20101101), 10 * 15 * 60}, + {YYYYMMDDToDay(20100328), YYYYMMDDToDay(20100330), 10 * 15 * 60} })) ); @@ -481,7 +481,7 @@ INSTANTIATE_TEST_SUITE_P(AllTimezones_Year1970_WHOLE, ::testing::ValuesIn(allTimezones(false)), ::testing::ValuesIn(std::initializer_list{ // Values from tests/date_lut3.cpp - {YYYYMMDDToDay(19700101), YYYYMMDDToDay(19701231), 3191 /*53m 11s*/}, + {YYYYMMDDToDay(19700101), YYYYMMDDToDay(19701231), 10 * 3191 /*53m 11s*/}, })) ); @@ -491,7 +491,7 @@ INSTANTIATE_TEST_SUITE_P(AllTimezones_Year2010_WHOLE, ::testing::ValuesIn(allTimezones(false)), ::testing::ValuesIn(std::initializer_list{ // Values from tests/date_lut3.cpp - {YYYYMMDDToDay(20100101), YYYYMMDDToDay(20101231), 3191 /*53m 11s*/}, + {YYYYMMDDToDay(20100101), YYYYMMDDToDay(20101231), 10 * 3191 /*53m 11s*/}, })) ); @@ -501,7 +501,7 @@ INSTANTIATE_TEST_SUITE_P(AllTimezones_Year2020_WHOLE, ::testing::ValuesIn(allTimezones()), ::testing::ValuesIn(std::initializer_list{ // Values from tests/date_lut3.cpp - {YYYYMMDDToDay(20200101), YYYYMMDDToDay(20201231), 3191 /*53m 11s*/}, + {YYYYMMDDToDay(20200101), YYYYMMDDToDay(20201231), 10 * 3191 /*53m 11s*/}, })) ); @@ -510,8 +510,8 @@ INSTANTIATE_TEST_SUITE_P(AllTimezones_PreEpoch, ::testing::Combine( ::testing::ValuesIn(allTimezones(false)), ::testing::ValuesIn(std::initializer_list{ - {YYYYMMDDToDay(19500101), YYYYMMDDToDay(19600101), 15 * 60}, - {YYYYMMDDToDay(19300101), YYYYMMDDToDay(19350101), 11 * 15 * 60} + {YYYYMMDDToDay(19500101), YYYYMMDDToDay(19600101), 10 * 15 * 60}, + {YYYYMMDDToDay(19300101), YYYYMMDDToDay(19350101), 10 * 11 * 15 * 60} })) ); @@ -520,8 +520,8 @@ INSTANTIATE_TEST_SUITE_P(AllTimezones_Year1970, ::testing::Combine( ::testing::ValuesIn(allTimezones(false)), ::testing::ValuesIn(std::initializer_list{ - {YYYYMMDDToDay(19700101), YYYYMMDDToDay(19700201), 15 * 60}, - {YYYYMMDDToDay(19700101), YYYYMMDDToDay(19701231), 11 * 13 * 17} + {YYYYMMDDToDay(19700101), YYYYMMDDToDay(19700201), 10 * 15 * 60}, + {YYYYMMDDToDay(19700101), YYYYMMDDToDay(19701231), 10 * 11 * 13 * 17} // // 11 was chosen as a number which can't divide product of 2-combinarions of (7, 24, 60), // // to reduce likelehood of hitting same hour/minute/second values for different days. // // + 12 is just to make sure that last day is covered fully. From 2b7d2fab82d7ef95602e397d8c025a53f97ecc3f Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 26 Mar 2021 19:40:02 +0300 Subject: [PATCH 21/34] less flaky functional tests --- tests/clickhouse-test | 6 ++-- ...4_performance_introspection_and_logging.sh | 2 +- .../00956_sensitive_data_masking.sh | 2 +- .../01231_log_queries_min_type.sql | 9 +++--- tests/queries/0_stateless/01304_direct_io.sh | 10 +++--- .../01344_min_bytes_to_use_mmap_io_index.sql | 2 +- ...terialized_view_with_join_on_query_log.sql | 4 +-- .../0_stateless/01475_read_subcolumns.sql | 8 ++--- .../01531_query_log_query_comment.sql | 4 +-- .../0_stateless/01533_multiple_nested.sql | 4 +-- ...1546_log_queries_min_query_duration_ms.sql | 12 +++---- .../01547_query_log_current_database.sql | 3 +- .../01548_query_log_query_execution_ms.sql | 3 +- .../0_stateless/01600_benchmark_query.sh | 9 +++--- ...01686_event_time_microseconds_part_log.sql | 2 ++ .../0_stateless/01702_system_query_log.sql | 2 +- tests/queries/skip_list.json | 31 ++++++++++--------- 17 files changed, 55 insertions(+), 58 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 908da85c1c1..1604c08dcb6 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -170,7 +170,7 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std if need_drop_database: clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) - seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 10) + seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 20) try: clickhouse_proc_create.communicate(("DROP DATABASE " + database), timeout=seconds_left) except TimeoutExpired: @@ -207,7 +207,7 @@ def need_retry(stderr): def get_processlist(args): try: clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - (stdout, _) = clickhouse_proc.communicate((b"SHOW PROCESSLIST FORMAT Vertical"), timeout=10) + (stdout, _) = clickhouse_proc.communicate((b"SHOW PROCESSLIST FORMAT Vertical"), timeout=20) return False, stdout.decode('utf-8') except Exception as ex: print("Exception", ex) @@ -352,7 +352,7 @@ def run_tests_array(all_tests_with_params): clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) failed_to_check = False try: - clickhouse_proc.communicate(("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite)), timeout=10) + clickhouse_proc.communicate(("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite)), timeout=20) except: failed_to_check = True diff --git a/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh b/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh index e51e4fea5db..cc5ece15435 100755 --- a/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh +++ b/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh @@ -48,7 +48,7 @@ SELECT threads_realtime >= threads_time_user_system_io, any(length(thread_ids)) >= 1 FROM - (SELECT * FROM system.query_log PREWHERE query='$heavy_cpu_query' WHERE event_date >= today()-1 AND current_database = currentDatabase() AND type=2 ORDER BY event_time DESC LIMIT 1) + (SELECT * FROM system.query_log PREWHERE query='$heavy_cpu_query' WHERE event_date >= today()-2 AND current_database = currentDatabase() AND type=2 ORDER BY event_time DESC LIMIT 1) ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV" # Clean diff --git a/tests/queries/0_stateless/00956_sensitive_data_masking.sh b/tests/queries/0_stateless/00956_sensitive_data_masking.sh index 764cb6a713e..6b9990ee819 100755 --- a/tests/queries/0_stateless/00956_sensitive_data_masking.sh +++ b/tests/queries/0_stateless/00956_sensitive_data_masking.sh @@ -97,7 +97,7 @@ echo 7 # and finally querylog $CLICKHOUSE_CLIENT \ --server_logs_file=/dev/null \ - --query="select * from system.query_log where current_database = currentDatabase() AND event_time > now() - 10 and query like '%TOPSECRET%';" + --query="select * from system.query_log where current_database = currentDatabase() AND event_date >= yesterday() and query like '%TOPSECRET%';" rm -f "$tmp_file" >/dev/null 2>&1 diff --git a/tests/queries/0_stateless/01231_log_queries_min_type.sql b/tests/queries/0_stateless/01231_log_queries_min_type.sql index 9659739b61d..382b7f0bf7e 100644 --- a/tests/queries/0_stateless/01231_log_queries_min_type.sql +++ b/tests/queries/0_stateless/01231_log_queries_min_type.sql @@ -2,18 +2,18 @@ set log_queries=1; select '01231_log_queries_min_type/QUERY_START'; system flush logs; -select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/QUERY_START%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute; +select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/QUERY_START%' and query not like '%system.query_log%' and event_date >= yesterday(); set log_queries_min_type='EXCEPTION_BEFORE_START'; select '01231_log_queries_min_type/EXCEPTION_BEFORE_START'; system flush logs; -select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_BEFORE_START%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute; +select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_BEFORE_START%' and query not like '%system.query_log%' and event_date >= yesterday(); set max_rows_to_read='100K'; set log_queries_min_type='EXCEPTION_WHILE_PROCESSING'; select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } system flush logs; -select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute and type = 'ExceptionWhileProcessing'; +select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' and query not like '%system.query_log%' and event_date >= yesterday() and type = 'ExceptionWhileProcessing'; select '01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } system flush logs; @@ -21,7 +21,6 @@ select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING%' and query not like '%system.query_log%' and - event_date = today() and - event_time >= now() - interval 1 minute and + event_date >= yesterday() and type = 'ExceptionWhileProcessing' and has(Settings.Names, 'max_rows_to_read'); diff --git a/tests/queries/0_stateless/01304_direct_io.sh b/tests/queries/0_stateless/01304_direct_io.sh index 3ba3d020d99..7505173ddba 100755 --- a/tests/queries/0_stateless/01304_direct_io.sh +++ b/tests/queries/0_stateless/01304_direct_io.sh @@ -9,12 +9,12 @@ $CLICKHOUSE_CLIENT --multiquery --query " CREATE TABLE bug (UserID UInt64, Date Date) ENGINE = MergeTree ORDER BY Date; INSERT INTO bug SELECT rand64(), '2020-06-07' FROM numbers(50000000); OPTIMIZE TABLE bug FINAL;" +LOG="$CLICKHOUSE_TMP/err-$CLICKHOUSE_DATABASE" +$CLICKHOUSE_BENCHMARK --iterations 10 --max_threads 100 --min_bytes_to_use_direct_io 1 <<< "SELECT sum(UserID) FROM bug PREWHERE NOT ignore(Date)" 1>/dev/null 2>"$LOG" +cat "$LOG" | grep Exception +cat "$LOG" | grep Loaded -$CLICKHOUSE_BENCHMARK --iterations 10 --max_threads 100 --min_bytes_to_use_direct_io 1 <<< "SELECT sum(UserID) FROM bug PREWHERE NOT ignore(Date)" 1>/dev/null 2>"$CLICKHOUSE_TMP"/err -cat "$CLICKHOUSE_TMP"/err | grep Exception -cat "$CLICKHOUSE_TMP"/err | grep Loaded - -rm "$CLICKHOUSE_TMP"/err +rm "$LOG" $CLICKHOUSE_CLIENT --multiquery --query " DROP TABLE bug;" diff --git a/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql b/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql index 9044ee08f8d..7aab991d203 100644 --- a/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql +++ b/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql @@ -6,6 +6,6 @@ SET min_bytes_to_use_mmap_io = 1; SELECT * FROM test_01344 WHERE x = 'Hello, world'; SYSTEM FLUSH LOGS; -SELECT PE.Values FROM system.query_log ARRAY JOIN ProfileEvents AS PE WHERE current_database = currentDatabase() AND event_date >= yesterday() AND event_time >= now() - 300 AND query LIKE 'SELECT * FROM test_01344 WHERE x = ''Hello, world''%' AND PE.Names = 'CreatedReadBufferMMap' AND type = 2 ORDER BY event_time DESC LIMIT 1; +SELECT PE.Values FROM system.query_log ARRAY JOIN ProfileEvents AS PE WHERE current_database = currentDatabase() AND event_date >= yesterday() AND query LIKE 'SELECT * FROM test_01344 WHERE x = ''Hello, world''%' AND PE.Names = 'CreatedReadBufferMMap' AND type = 2 ORDER BY event_time DESC LIMIT 1; DROP TABLE test_01344; diff --git a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql index 950d4fe097f..3380f04f8c9 100644 --- a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql +++ b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql @@ -17,7 +17,7 @@ CREATE MATERIALIZED VIEW slow_log Engine=Memory AS extract(query,'/\\*\\s*QUERY_GROUP_ID:(.*?)\\s*\\*/') as QUERY_GROUP_ID, * FROM system.query_log - WHERE type<>1 and event_date >= yesterday() and event_time > now() - 120 + WHERE type<>1 and event_date >= yesterday() ) as ql INNER JOIN expected_times USING (QUERY_GROUP_ID) WHERE query_duration_ms > max_query_duration_ms @@ -38,7 +38,7 @@ SELECT extract(query,'/\\*\\s*QUERY_GROUP_ID:(.*?)\\s*\\*/') as QUERY_GROUP_ID, count() FROM system.query_log -WHERE current_database = currentDatabase() AND type<>1 and event_date >= yesterday() and event_time > now() - 20 and QUERY_GROUP_ID<>'' +WHERE current_database = currentDatabase() AND type<>1 and event_date >= yesterday() and QUERY_GROUP_ID<>'' GROUP BY QUERY_GROUP_ID ORDER BY QUERY_GROUP_ID; diff --git a/tests/queries/0_stateless/01475_read_subcolumns.sql b/tests/queries/0_stateless/01475_read_subcolumns.sql index 16832c4fc59..3457d17dba1 100644 --- a/tests/queries/0_stateless/01475_read_subcolumns.sql +++ b/tests/queries/0_stateless/01475_read_subcolumns.sql @@ -10,7 +10,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] FROM system.query_log WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT a.size0 FROM %t_arr%')) - AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + AND current_database = currentDatabase(); SELECT '====tuple===='; DROP TABLE IF EXISTS t_tup; @@ -27,7 +27,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] FROM system.query_log WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT t._ FROM %t_tup%')) - AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + AND current_database = currentDatabase(); SELECT '====nullable===='; DROP TABLE IF EXISTS t_nul; @@ -41,7 +41,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] FROM system.query_log WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT n.null FROM %t_nul%')) - AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + AND current_database = currentDatabase(); SELECT '====map===='; SET allow_experimental_map_type = 1; @@ -60,7 +60,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] FROM system.query_log WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT m.% FROM %t_map%')) - AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + AND current_database = currentDatabase(); DROP TABLE t_arr; DROP TABLE t_nul; diff --git a/tests/queries/0_stateless/01531_query_log_query_comment.sql b/tests/queries/0_stateless/01531_query_log_query_comment.sql index 2e1faf1b9e4..bed3a0fd95e 100644 --- a/tests/queries/0_stateless/01531_query_log_query_comment.sql +++ b/tests/queries/0_stateless/01531_query_log_query_comment.sql @@ -5,7 +5,7 @@ set enable_global_with_statement=1; select /* test=01531, enable_global_with_statement=0 */ 2; system flush logs; select count() from system.query_log -where event_time >= now() - interval 5 minute +where event_date >= yesterday() and query like '%select /* test=01531, enable_global_with_statement=0 */ 2%' and current_database = currentDatabase() ; @@ -14,7 +14,7 @@ set enable_global_with_statement=1; select /* test=01531 enable_global_with_statement=1 */ 2; system flush logs; select count() from system.query_log -where event_time >= now() - interval 5 minute +where event_date >= yesterday() and query like '%select /* test=01531 enable_global_with_statement=1 */ 2%' and current_database = currentDatabase() ; diff --git a/tests/queries/0_stateless/01533_multiple_nested.sql b/tests/queries/0_stateless/01533_multiple_nested.sql index 38c80617334..1c6722268ac 100644 --- a/tests/queries/0_stateless/01533_multiple_nested.sql +++ b/tests/queries/0_stateless/01533_multiple_nested.sql @@ -36,7 +36,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] FROM system.query_log WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT col1.a FROM %nested%')) - AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + AND event_date >= yesterday() AND current_database = currentDatabase(); SYSTEM DROP MARK CACHE; SELECT col3.n2.s FROM nested FORMAT Null; @@ -46,7 +46,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] FROM system.query_log WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT col3.n2.s FROM %nested%')) - AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + AND event_date >= yesterday() AND current_database = currentDatabase(); DROP TABLE nested; diff --git a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql index f0f681288cf..e551942aeb5 100644 --- a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql +++ b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql @@ -15,16 +15,14 @@ where query like '%01546_log_queries_min_query_duration_ms-fast%' and query not like '%system.query_log%' and current_database = currentDatabase() - and event_date = today() - and event_time >= now() - interval 1 minute; + and event_date >= yesterday(); select count() from system.query_thread_log where query like '%01546_log_queries_min_query_duration_ms-fast%' and query not like '%system.query_thread_log%' and current_database = currentDatabase() - and event_date = today() - and event_time >= now() - interval 1 minute; + and event_date >= yesterday(); -- -- slow -- query logged @@ -40,8 +38,7 @@ where query like '%01546_log_queries_min_query_duration_ms-slow%' and query not like '%system.query_log%' and current_database = currentDatabase() - and event_date = today() - and event_time >= now() - interval 1 minute; + and event_date >= yesterday(); -- There at least two threads involved in a simple query -- (one thread just waits another, sigh) select count() == 2 @@ -50,5 +47,4 @@ where query like '%01546_log_queries_min_query_duration_ms-slow%' and query not like '%system.query_thread_log%' and current_database = currentDatabase() - and event_date = today() - and event_time >= now() - interval 1 minute; + and event_date >= yesterday(); diff --git a/tests/queries/0_stateless/01547_query_log_current_database.sql b/tests/queries/0_stateless/01547_query_log_current_database.sql index c0ad22163ba..755df2fab41 100644 --- a/tests/queries/0_stateless/01547_query_log_current_database.sql +++ b/tests/queries/0_stateless/01547_query_log_current_database.sql @@ -23,8 +23,7 @@ from system.query_log where query like '%01547_query_log_current_database%' and current_database = currentDatabase() - and event_date = today() - and event_time >= now() - interval 1 minute; + and event_date >= yesterday(); -- at least two threads for processing -- (but one just waits for another, sigh) diff --git a/tests/queries/0_stateless/01548_query_log_query_execution_ms.sql b/tests/queries/0_stateless/01548_query_log_query_execution_ms.sql index e80e84646be..028b3400d01 100644 --- a/tests/queries/0_stateless/01548_query_log_query_execution_ms.sql +++ b/tests/queries/0_stateless/01548_query_log_query_execution_ms.sql @@ -13,8 +13,7 @@ where query like '%01548_query_log_query_execution_ms%' and current_database = currentDatabase() and query_duration_ms between 100 and 800 - and event_date = today() - and event_time >= now() - interval 1 minute; + and event_date >= yesterday(); -- at least two threads for processing -- (but one just waits for another, sigh) diff --git a/tests/queries/0_stateless/01600_benchmark_query.sh b/tests/queries/0_stateless/01600_benchmark_query.sh index a563c87a10f..1cf9cb23c3c 100755 --- a/tests/queries/0_stateless/01600_benchmark_query.sh +++ b/tests/queries/0_stateless/01600_benchmark_query.sh @@ -4,9 +4,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_BENCHMARK --iterations 10 --query "SELECT 1" 1>/dev/null 2>"$CLICKHOUSE_TMP"/err +LOG="$CLICKHOUSE_TMP/err-$CLICKHOUSE_DATABASE" +$CLICKHOUSE_BENCHMARK --iterations 10 --query "SELECT 1" 1>/dev/null 2>"$LOG" -cat "$CLICKHOUSE_TMP"/err | grep Exception -cat "$CLICKHOUSE_TMP"/err | grep Loaded +cat "$LOG" | grep Exception +cat "$LOG" | grep Loaded -rm "$CLICKHOUSE_TMP"/err +rm "$LOG" diff --git a/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql b/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql index a1b419527d4..1f709f3c0c0 100644 --- a/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql +++ b/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql @@ -15,6 +15,8 @@ SYSTEM FLUSH LOGS; WITH ( SELECT (event_time, event_time_microseconds) FROM system.part_log + WHERE "table" = 'table_with_single_pk' + AND "database" = currentDatabase() ORDER BY event_time DESC LIMIT 1 ) AS time diff --git a/tests/queries/0_stateless/01702_system_query_log.sql b/tests/queries/0_stateless/01702_system_query_log.sql index f31d8de3577..5c3de9cf912 100644 --- a/tests/queries/0_stateless/01702_system_query_log.sql +++ b/tests/queries/0_stateless/01702_system_query_log.sql @@ -136,7 +136,7 @@ SELECT 'ACTUAL LOG CONTENT:'; -- Try to filter out all possible previous junk events by excluding old log entries, SELECT query_kind, query FROM system.query_log WHERE - log_comment LIKE '%system.query_log%' AND type == 'QueryStart' AND event_time >= now() - 10 + log_comment LIKE '%system.query_log%' AND type == 'QueryStart' AND event_date >= yesterday() AND current_database == currentDatabase() ORDER BY event_time_microseconds; diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 77d4a9b8499..3b92dc6c064 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -525,8 +525,8 @@ "00571_non_exist_database_when_create_materializ_view", "00575_illegal_column_exception_when_drop_depen_column", "00599_create_view_with_subquery", - "00604_show_create_database", "00600_replace_running_query", + "00604_show_create_database", "00612_http_max_query_size", "00619_union_highlite", "00620_optimize_on_nonleader_replica_zookeeper", @@ -577,6 +577,7 @@ "00933_test_fix_extra_seek_on_compressed_cache", "00933_ttl_replicated_zookeeper", "00933_ttl_with_default", + "00950_dict_get", "00955_test_final_mark", "00976_ttl_with_old_parts", "00980_merge_alter_settings", @@ -740,8 +741,8 @@ "01530_drop_database_atomic_sync", "01541_max_memory_usage_for_user_long", "01542_dictionary_load_exception_race", - "01560_optimize_on_insert_zookeeper", "01545_system_errors", // looks at the difference of values in system.errors + "01560_optimize_on_insert_zookeeper", "01575_disable_detach_table_of_dictionary", "01593_concurrent_alter_mutations_kill", "01593_concurrent_alter_mutations_kill_many_replicas", @@ -754,11 +755,23 @@ "01603_rename_overwrite_bug", "01646_system_restart_replicas_smoke", // system restart replicas is a global query "01656_test_query_log_factories_info", + "01658_read_file_to_stringcolumn", "01669_columns_declaration_serde", "01676_dictget_in_default_expression", + "01681_cache_dictionary_simple_key", + "01682_cache_dictionary_complex_key", + "01683_flat_dictionary", + "01684_ssd_cache_dictionary_simple_key", + "01685_ssd_cache_dictionary_complex_key", "01700_system_zookeeper_path_in", + "01702_system_query_log", // It's ok to execute in parallel with oter tests but not several instances of the same test. + "01702_system_query_log", // Runs many global system queries "01715_background_checker_blather_zookeeper", + "01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test. "01747_alter_partition_key_enum_zookeeper", + "01748_dictionary_table_dot", // creates database + "01760_polygon_dictionaries", + "01760_system_dictionaries", "01761_alter_decimal_zookeeper", "attach", "ddl_dictionaries", @@ -767,18 +780,6 @@ "live_view", "memory_leak", "memory_limit", - "polygon_dicts", // they use an explicitly specified database - "01658_read_file_to_stringcolumn", - "01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test. - "01702_system_query_log", // It's ok to execute in parallel with oter tests but not several instances of the same test. - "01748_dictionary_table_dot", // creates database - "00950_dict_get", - "01683_flat_dictionary", - "01681_cache_dictionary_simple_key", - "01682_cache_dictionary_complex_key", - "01684_ssd_cache_dictionary_simple_key", - "01685_ssd_cache_dictionary_complex_key", - "01760_system_dictionaries", - "01760_polygon_dictionaries" + "polygon_dicts" // they use an explicitly specified database ] } From 19d4be7c01c98e1d55d54e6114bcfe0e148b9bf9 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 26 Mar 2021 22:30:30 +0300 Subject: [PATCH 22/34] fixes --- .../01231_log_queries_min_type.sql | 15 +++++++++----- ...ference => 01304_direct_io_long.reference} | 0 ...4_direct_io.sh => 01304_direct_io_long.sh} | 0 .../01533_multiple_nested.reference | 20 +++++++++---------- .../0_stateless/01533_multiple_nested.sql | 2 +- ...og_queries_min_query_duration_ms.reference | 2 +- ...1546_log_queries_min_query_duration_ms.sql | 14 +++++-------- 7 files changed, 27 insertions(+), 26 deletions(-) rename tests/queries/0_stateless/{01304_direct_io.reference => 01304_direct_io_long.reference} (100%) rename tests/queries/0_stateless/{01304_direct_io.sh => 01304_direct_io_long.sh} (100%) diff --git a/tests/queries/0_stateless/01231_log_queries_min_type.sql b/tests/queries/0_stateless/01231_log_queries_min_type.sql index 382b7f0bf7e..66a3a6b967c 100644 --- a/tests/queries/0_stateless/01231_log_queries_min_type.sql +++ b/tests/queries/0_stateless/01231_log_queries_min_type.sql @@ -2,25 +2,30 @@ set log_queries=1; select '01231_log_queries_min_type/QUERY_START'; system flush logs; -select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/QUERY_START%' and query not like '%system.query_log%' and event_date >= yesterday(); +select count() from system.query_log where current_database = currentDatabase() + and query like 'select \'01231_log_queries_min_type/QUERY_START%' + and event_date >= yesterday(); set log_queries_min_type='EXCEPTION_BEFORE_START'; select '01231_log_queries_min_type/EXCEPTION_BEFORE_START'; system flush logs; -select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_BEFORE_START%' and query not like '%system.query_log%' and event_date >= yesterday(); +select count() from system.query_log where current_database = currentDatabase() + and query like 'select \'01231_log_queries_min_type/EXCEPTION_BEFORE_START%' + and event_date >= yesterday(); set max_rows_to_read='100K'; set log_queries_min_type='EXCEPTION_WHILE_PROCESSING'; select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } system flush logs; -select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' and query not like '%system.query_log%' and event_date >= yesterday() and type = 'ExceptionWhileProcessing'; +select count() from system.query_log where current_database = currentDatabase() + and query like 'select \'01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' + and event_date >= yesterday() and type = 'ExceptionWhileProcessing'; select '01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } system flush logs; select count() from system.query_log where current_database = currentDatabase() and - query like '%01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING%' and - query not like '%system.query_log%' and + query like 'select \'01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING%' and event_date >= yesterday() and type = 'ExceptionWhileProcessing' and has(Settings.Names, 'max_rows_to_read'); diff --git a/tests/queries/0_stateless/01304_direct_io.reference b/tests/queries/0_stateless/01304_direct_io_long.reference similarity index 100% rename from tests/queries/0_stateless/01304_direct_io.reference rename to tests/queries/0_stateless/01304_direct_io_long.reference diff --git a/tests/queries/0_stateless/01304_direct_io.sh b/tests/queries/0_stateless/01304_direct_io_long.sh similarity index 100% rename from tests/queries/0_stateless/01304_direct_io.sh rename to tests/queries/0_stateless/01304_direct_io_long.sh diff --git a/tests/queries/0_stateless/01533_multiple_nested.reference b/tests/queries/0_stateless/01533_multiple_nested.reference index ba37ce1c32c..138a7cfd7f2 100644 --- a/tests/queries/0_stateless/01533_multiple_nested.reference +++ b/tests/queries/0_stateless/01533_multiple_nested.reference @@ -13,16 +13,16 @@ col3 read files 4 6 -0 899984 7199412 -1 899987 7199877 -2 899990 7200255 -3 899993 7199883 -4 899996 7199798 -5 899999 7200306 -6 900002 7200064 -7 900005 7199429 -8 900008 7200067 -9 899992 7199993 +0 89982 719752 +1 89988 720017 +2 89994 720152 +3 90000 720157 +4 90006 720100 +5 90012 720168 +6 90018 720106 +7 90005 719891 +8 89992 719854 +9 89979 719706 0 [] 0 [0] 1 [0,2] diff --git a/tests/queries/0_stateless/01533_multiple_nested.sql b/tests/queries/0_stateless/01533_multiple_nested.sql index 1c6722268ac..0ddb0cfbfb4 100644 --- a/tests/queries/0_stateless/01533_multiple_nested.sql +++ b/tests/queries/0_stateless/01533_multiple_nested.sql @@ -59,7 +59,7 @@ ENGINE = MergeTree ORDER BY id SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO nested SELECT number, arrayMap(x -> (x, arrayMap(y -> (toString(y * x), y + x), range(number % 17))), range(number % 19)) FROM numbers(1000000); +INSERT INTO nested SELECT number, arrayMap(x -> (x, arrayMap(y -> (toString(y * x), y + x), range(number % 17))), range(number % 19)) FROM numbers(100000); SELECT id % 10, sum(length(col1)), sumArray(arrayMap(x -> length(x), col1.n.b)) FROM nested GROUP BY id % 10; SELECT arraySum(col1.a), arrayMap(x -> x * x * 2, col1.a) FROM nested ORDER BY id LIMIT 5; diff --git a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.reference b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.reference index 0463db26710..8947c3c2bc3 100644 --- a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.reference +++ b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.reference @@ -1,4 +1,4 @@ 0 0 1 -1 +OK diff --git a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql index e551942aeb5..20854da0e8a 100644 --- a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql +++ b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql @@ -12,15 +12,13 @@ system flush logs; select count() from system.query_log where - query like '%01546_log_queries_min_query_duration_ms-fast%' - and query not like '%system.query_log%' + query like 'select \'01546_log_queries_min_query_duration_ms-fast%' and current_database = currentDatabase() and event_date >= yesterday(); select count() from system.query_thread_log where - query like '%01546_log_queries_min_query_duration_ms-fast%' - and query not like '%system.query_thread_log%' + query like 'select \'01546_log_queries_min_query_duration_ms-fast%' and current_database = currentDatabase() and event_date >= yesterday(); @@ -35,16 +33,14 @@ system flush logs; select count() from system.query_log where - query like '%01546_log_queries_min_query_duration_ms-slow%' - and query not like '%system.query_log%' + query like 'select \'01546_log_queries_min_query_duration_ms-slow%' and current_database = currentDatabase() and event_date >= yesterday(); -- There at least two threads involved in a simple query -- (one thread just waits another, sigh) -select count() == 2 +select if(count() == 2, 'OK', 'Fail: ' || toString(count())) from system.query_thread_log where - query like '%01546_log_queries_min_query_duration_ms-slow%' - and query not like '%system.query_thread_log%' + query like 'select \'01546_log_queries_min_query_duration_ms-slow%' and current_database = currentDatabase() and event_date >= yesterday(); From 605ce6d5fd490bbe49ab6589ef3e776922d09ab3 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 14:39:27 +0300 Subject: [PATCH 23/34] fix --- tests/queries/0_stateless/01231_log_queries_min_type.sql | 1 + .../queries/0_stateless/01531_query_log_query_comment.sql | 8 ++++---- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01231_log_queries_min_type.sql b/tests/queries/0_stateless/01231_log_queries_min_type.sql index 66a3a6b967c..b6e27506b72 100644 --- a/tests/queries/0_stateless/01231_log_queries_min_type.sql +++ b/tests/queries/0_stateless/01231_log_queries_min_type.sql @@ -16,6 +16,7 @@ select count() from system.query_log where current_database = currentDatabase() set max_rows_to_read='100K'; set log_queries_min_type='EXCEPTION_WHILE_PROCESSING'; select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } +set max_rows_to_read=0; system flush logs; select count() from system.query_log where current_database = currentDatabase() and query like 'select \'01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' diff --git a/tests/queries/0_stateless/01531_query_log_query_comment.sql b/tests/queries/0_stateless/01531_query_log_query_comment.sql index bed3a0fd95e..19d6eac0e15 100644 --- a/tests/queries/0_stateless/01531_query_log_query_comment.sql +++ b/tests/queries/0_stateless/01531_query_log_query_comment.sql @@ -1,20 +1,20 @@ set log_queries=1; set log_queries_min_type='QUERY_FINISH'; -set enable_global_with_statement=1; +set enable_global_with_statement=0; select /* test=01531, enable_global_with_statement=0 */ 2; system flush logs; select count() from system.query_log where event_date >= yesterday() - and query like '%select /* test=01531, enable_global_with_statement=0 */ 2%' + and query like 'select /* test=01531, enable_global_with_statement=0 */ 2%' and current_database = currentDatabase() ; set enable_global_with_statement=1; -select /* test=01531 enable_global_with_statement=1 */ 2; +select /* test=01531, enable_global_with_statement=1 */ 2; system flush logs; select count() from system.query_log where event_date >= yesterday() - and query like '%select /* test=01531 enable_global_with_statement=1 */ 2%' + and query like 'select /* test=01531, enable_global_with_statement=1 */ 2%' and current_database = currentDatabase() ; From 23198059f3995bb657262c73aee67a45e7a6eade Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 14:47:57 +0300 Subject: [PATCH 24/34] save query log --- docker/test/stateless/run.sh | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 3119ae27c59..853814faae8 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -74,7 +74,13 @@ timeout "$MAX_RUN_TIME" bash -c run_tests ||: ./process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv -pigz < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.gz ||: +clickhouse-client -q "sytem flush logs" ||: + +pigz < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.gz & +clickhouse-client -q "select * from system.query_log format TSVWithNamesAndTypes" | pigz > /test_output/query-log.tsv.gz & +clickhouse-client -q "select * from system.query_thread_log format TSVWithNamesAndTypes" | pigz > /test_output/query-thread-log.tsv.gz & +wait ||: + mv /var/log/clickhouse-server/stderr.log /test_output/ ||: if [[ -n "$WITH_COVERAGE" ]] && [[ "$WITH_COVERAGE" -eq 1 ]]; then tar -chf /test_output/clickhouse_coverage.tar.gz /profraw ||: From 7813ac7d2b75354f51322ff57052b13158fb9ee6 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 16:11:21 +0300 Subject: [PATCH 25/34] fix --- tests/queries/0_stateless/01231_log_queries_min_type.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01231_log_queries_min_type.sql b/tests/queries/0_stateless/01231_log_queries_min_type.sql index b6e27506b72..b3540f3354b 100644 --- a/tests/queries/0_stateless/01231_log_queries_min_type.sql +++ b/tests/queries/0_stateless/01231_log_queries_min_type.sql @@ -22,8 +22,10 @@ select count() from system.query_log where current_database = currentDatabase() and query like 'select \'01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' and event_date >= yesterday() and type = 'ExceptionWhileProcessing'; +set max_rows_to_read='100K'; select '01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } system flush logs; +set max_rows_to_read=0; select count() from system.query_log where current_database = currentDatabase() and query like 'select \'01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING%' and From c711a6c5fa1b97994a70b4d281ea987a4cf9ebb0 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 19:25:13 +0300 Subject: [PATCH 26/34] make it faster --- .../0_stateless/01686_event_time_microseconds_part_log.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql b/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql index 1f709f3c0c0..4a653379ef1 100644 --- a/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql +++ b/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql @@ -8,7 +8,7 @@ CREATE TABLE table_with_single_pk ENGINE = MergeTree ORDER BY key; -INSERT INTO table_with_single_pk SELECT number, toString(number % 10) FROM numbers(10000000); +INSERT INTO table_with_single_pk SELECT number, toString(number % 10) FROM numbers(1000000); SYSTEM FLUSH LOGS; From 47cc8938f925dd7911777ddfe813151e4e228961 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 19:26:50 +0300 Subject: [PATCH 27/34] faster --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 1604c08dcb6..5a8899e2658 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -448,7 +448,7 @@ def run_tests_array(all_tests_with_params): failures_chain += 1 status += MSG_FAIL status += print_test_time(total_time) - status += " - Long test not marked as 'long'" + status += " - Test runs too long (> 30s). Make it faster." else: passed_total += 1 failures_chain = 0 From 653b930e542f2ea03bf648a1cd41b30bb73b3f1a Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 21:24:29 +0300 Subject: [PATCH 28/34] random order in flaky check --- docker/test/stateless/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 853814faae8..0781e346bb2 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -51,6 +51,7 @@ function run_tests() # Skip these tests, because they fail when we rerun them multiple times if [ "$NUM_TRIES" -gt "1" ]; then + ADDITIONAL_OPTIONS+=('--order=random') ADDITIONAL_OPTIONS+=('--skip') ADDITIONAL_OPTIONS+=('00000_no_tests_to_skip') ADDITIONAL_OPTIONS+=('--jobs') From 43d9972f18d43c6736c08e67c01deb64b733109a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 30 Mar 2021 01:34:02 +0300 Subject: [PATCH 29/34] remove useless code --- .../AggregateFunctionWindowFunnel.h | 20 +++++-------------- 1 file changed, 5 insertions(+), 15 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index e4a275555bf..9644b8d9ea4 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -19,15 +19,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -struct ComparePair final -{ - template - bool operator()(const std::pair & lhs, const std::pair & rhs) const - { - return lhs.first == rhs.first ? lhs.second < rhs.second : lhs.first < rhs.first; - } -}; - static constexpr auto max_events = 32; template @@ -35,7 +26,6 @@ struct AggregateFunctionWindowFunnelData { using TimestampEvent = std::pair; using TimestampEvents = PODArrayWithStackMemory; - using Comparator = ComparePair; bool sorted = true; TimestampEvents events_list; @@ -69,7 +59,7 @@ struct AggregateFunctionWindowFunnelData /// either sort whole container or do so partially merging ranges afterwards if (!sorted && !other.sorted) - std::stable_sort(std::begin(events_list), std::end(events_list), Comparator{}); + std::stable_sort(std::begin(events_list), std::end(events_list)); else { const auto begin = std::begin(events_list); @@ -77,12 +67,12 @@ struct AggregateFunctionWindowFunnelData const auto end = std::end(events_list); if (!sorted) - std::stable_sort(begin, middle, Comparator{}); + std::stable_sort(begin, middle); if (!other.sorted) - std::stable_sort(middle, end, Comparator{}); + std::stable_sort(middle, end); - std::inplace_merge(begin, middle, end, Comparator{}); + std::inplace_merge(begin, middle, end); } sorted = true; @@ -92,7 +82,7 @@ struct AggregateFunctionWindowFunnelData { if (!sorted) { - std::stable_sort(std::begin(events_list), std::end(events_list), Comparator{}); + std::stable_sort(std::begin(events_list), std::end(events_list)); sorted = true; } } From 6bd483f9d47bbe531b98ae9be5dcd1d66f2bd137 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 30 Mar 2021 03:39:33 +0300 Subject: [PATCH 30/34] individual tmp dir matching the database for each test run --- tests/clickhouse-test | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 5a8899e2658..b3ebe3a82e7 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1,5 +1,6 @@ #!/usr/bin/env python3 +import shutil import sys import os import os.path @@ -112,13 +113,14 @@ def get_db_engine(args, database_name): return " ENGINE=" + args.db_engine return "" # Will use default engine -def run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file): +def run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file, suite_tmp_dir): # print(client_options) start_time = datetime.now() if args.database: database = args.database os.environ.setdefault("CLICKHOUSE_DATABASE", database) + os.environ.setdefault("CLICKHOUSE_TMP", suite_tmp_dir) else: # If --database is not specified, we will create temporary database with unique name @@ -136,6 +138,12 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std return clickhouse_proc_create, "", "Timeout creating database {} before test".format(database), total_time os.environ["CLICKHOUSE_DATABASE"] = database + # Set temporary directory to match the randomly generated database, + # because .sh tests also use it for temporary files and we want to avoid + # collisions. + test_tmp_dir = os.path.join(suite_tmp_dir, database) + os.mkdir(test_tmp_dir) + os.environ.setdefault("CLICKHOUSE_TMP", test_tmp_dir) # This is for .sh tests os.environ["CLICKHOUSE_LOG_COMMENT"] = case_file @@ -185,6 +193,8 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std total_time = (datetime.now() - start_time).total_seconds() return clickhouse_proc_create, "", "Timeout dropping database {} after test".format(database), total_time + shutil.rmtree(test_tmp_dir) + total_time = (datetime.now() - start_time).total_seconds() # Normalize randomized database names in stdout, stderr files. @@ -367,7 +377,7 @@ def run_tests_array(all_tests_with_params): stdout_file = os.path.join(suite_tmp_dir, name) + file_suffix + '.stdout' stderr_file = os.path.join(suite_tmp_dir, name) + file_suffix + '.stderr' - proc, stdout, stderr, total_time = run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file) + proc, stdout, stderr, total_time = run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file, suite_tmp_dir) if proc.returncode is None: try: @@ -385,7 +395,7 @@ def run_tests_array(all_tests_with_params): else: counter = 1 while proc.returncode != 0 and need_retry(stderr): - proc, stdout, stderr, total_time = run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file) + proc, stdout, stderr, total_time = run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file, suite_tmp_dir) sleep(2**counter) counter += 1 if counter > 6: @@ -649,7 +659,6 @@ def main(args): os.environ.setdefault("CLICKHOUSE_CONFIG", args.configserver) if args.configclient: os.environ.setdefault("CLICKHOUSE_CONFIG_CLIENT", args.configclient) - os.environ.setdefault("CLICKHOUSE_TMP", tmp_dir) # Force to print server warnings in stderr # Shell scripts could change logging level From b136625307f9ac0402143efab15852a1e1e48caf Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 23 Jan 2021 12:18:15 +0300 Subject: [PATCH 31/34] Remove outdated suppressions, part 2 --- tests/msan_suppressions.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/msan_suppressions.txt b/tests/msan_suppressions.txt index e0edd7f3dd7..7ade6e04268 100644 --- a/tests/msan_suppressions.txt +++ b/tests/msan_suppressions.txt @@ -9,7 +9,6 @@ fun:tolower # Ideally, we should report these upstream. src:*/contrib/zlib-ng/* src:*/contrib/simdjson/* -src:*/contrib/lz4/* # Hyperscan fun:roseRunProgram From e9e4e0e49014fde5d8126584c9c68e06f98e327b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Mar 2021 04:07:57 +0300 Subject: [PATCH 32/34] Maybe fix false MSan report in GRPC --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index 7436366ceb3..8d558f03fe3 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 7436366ceb341ba5c00ea29f1645e02a2b70bf93 +Subproject commit 8d558f03fe370240081424fafa76cdc9301ea14b From bd5ddf052436cc00a9eeb5cae78758413552e7ef Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 30 Mar 2021 13:49:43 +0300 Subject: [PATCH 33/34] fixes --- docker/test/stateless/run.sh | 2 +- .../0_stateless/01547_query_log_current_database.sql | 7 +++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 0781e346bb2..61153ce3864 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -75,7 +75,7 @@ timeout "$MAX_RUN_TIME" bash -c run_tests ||: ./process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv -clickhouse-client -q "sytem flush logs" ||: +clickhouse-client -q "system flush logs" ||: pigz < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.gz & clickhouse-client -q "select * from system.query_log format TSVWithNamesAndTypes" | pigz > /test_output/query-log.tsv.gz & diff --git a/tests/queries/0_stateless/01547_query_log_current_database.sql b/tests/queries/0_stateless/01547_query_log_current_database.sql index 755df2fab41..5eec8a81ccc 100644 --- a/tests/queries/0_stateless/01547_query_log_current_database.sql +++ b/tests/queries/0_stateless/01547_query_log_current_database.sql @@ -21,7 +21,7 @@ system flush logs; select count() from system.query_log where - query like '%01547_query_log_current_database%' + query like 'select \'01547_query_log_current_database%' and current_database = currentDatabase() and event_date >= yesterday(); @@ -30,7 +30,6 @@ where select count() == 2 from system.query_thread_log where - query like '%01547_query_log_current_database%' + query like 'select \'01547_query_log_current_database%' and current_database = currentDatabase() - and event_date = today() - and event_time >= now() - interval 1 minute; + and event_date >= yesterday() From 64bfdb69d04ea9c7ce3a51d2585e87e2ef82ff3d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Mar 2021 15:24:22 +0300 Subject: [PATCH 34/34] Adjust perf test --- tests/performance/mmap_io.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/mmap_io.xml b/tests/performance/mmap_io.xml index d9e6037c2d2..b8f3f6e69dd 100644 --- a/tests/performance/mmap_io.xml +++ b/tests/performance/mmap_io.xml @@ -11,7 +11,7 @@ INSERT INTO hits_none SELECT WatchID FROM test.hits OPTIMIZE TABLE hits_none FINAL - + DROP TABLE hits_none