From 3fe2e1084bb3a7cc0def8b9b720b32455d2f10e7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 5 Sep 2023 22:25:34 +0000 Subject: [PATCH 01/93] test_read_equally_from_each_replica + prefer_localhost_replica=1 --- .../test.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py index c9b424c74d4..80104d77a9f 100644 --- a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py +++ b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py @@ -95,7 +95,14 @@ def create_tables(cluster, table_name): return "60\t0\t59\t1770\n" -def test_read_equally_from_each_replica(start_cluster): +@pytest.mark.parametrize( + "prefer_localhost_replica", + [ + pytest.param(0), + pytest.param(1), + ] +) +def test_read_equally_from_each_replica(start_cluster, prefer_localhost_replica): """create and populate table in special way (see create_table()), so parallel replicas will read equal number of rows from each replica """ @@ -110,7 +117,7 @@ def test_read_equally_from_each_replica(start_cluster): f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", settings={ "allow_experimental_parallel_reading_from_replicas": 2, - "prefer_localhost_replica": 0, + "prefer_localhost_replica": prefer_localhost_replica, "max_parallel_replicas": 3, "use_hedged_requests": 0, }, From 87a59e7084696d69871d8874c8a9a4ce66fa353d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 5 Sep 2023 23:12:35 +0000 Subject: [PATCH 02/93] Automatic style fix --- .../test_parallel_replicas_distributed_read_from_all/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py index 80104d77a9f..482192668f4 100644 --- a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py +++ b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py @@ -100,7 +100,7 @@ def create_tables(cluster, table_name): [ pytest.param(0), pytest.param(1), - ] + ], ) def test_read_equally_from_each_replica(start_cluster, prefer_localhost_replica): """create and populate table in special way (see create_table()), From 9881e75c05142813f2195997a23d6566c61bb075 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Sep 2023 11:28:52 +0000 Subject: [PATCH 03/93] Create read from remote step if there are remote replicas --- src/Interpreters/Cluster.cpp | 8 +------- src/Interpreters/Cluster.h | 1 - src/Interpreters/ClusterProxy/SelectStreamFactory.cpp | 8 +++++++- 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 891586d88b6..bb161702401 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -420,8 +420,6 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, if (address.is_local) info.local_addresses.push_back(address); - info.all_addresses.push_back(address); - auto pool = ConnectionPoolFactory::instance().get( static_cast(settings.distributed_connections_pool_size), address.host_name, address.port, @@ -564,7 +562,6 @@ void Cluster::addShard(const Settings & settings, Addresses && addresses, bool t ShardInfoInsertPathForInternalReplication && insert_paths, UInt32 weight, bool internal_replication) { Addresses shard_local_addresses; - Addresses shard_all_addresses; ConnectionPoolPtrs all_replicas_pools; all_replicas_pools.reserve(addresses.size()); @@ -582,7 +579,6 @@ void Cluster::addShard(const Settings & settings, Addresses && addresses, bool t all_replicas_pools.emplace_back(replica_pool); if (replica.is_local && !treat_local_as_remote) shard_local_addresses.push_back(replica); - shard_all_addresses.push_back(replica); } ConnectionPoolWithFailoverPtr shard_pool = std::make_shared( all_replicas_pools, settings.load_balancing, @@ -596,7 +592,6 @@ void Cluster::addShard(const Settings & settings, Addresses && addresses, bool t current_shard_num, weight, std::move(shard_local_addresses), - std::move(shard_all_addresses), std::move(shard_pool), std::move(all_replicas_pools), internal_replication @@ -647,6 +642,7 @@ void Cluster::initMisc() std::unique_ptr Cluster::getClusterWithReplicasAsShards(const Settings & settings, size_t max_replicas_from_shard) const { + LOG_DEBUG(&Poco::Logger::get(__FUNCTION__), "max_replicas_from_shard={}\n{}", max_replicas_from_shard, StackTrace().toString()); return std::unique_ptr{ new Cluster(ReplicasAsShardsTag{}, *this, settings, max_replicas_from_shard)}; } @@ -720,8 +716,6 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti if (address.is_local) info.local_addresses.push_back(address); - info.all_addresses.push_back(address); - auto pool = ConnectionPoolFactory::instance().get( static_cast(settings.distributed_connections_pool_size), address.host_name, diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index cb75487cbbc..b2bc03dd74d 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -217,7 +217,6 @@ public: UInt32 shard_num = 0; UInt32 weight = 1; Addresses local_addresses; - Addresses all_addresses; /// nullptr if there are no remote addresses ConnectionPoolWithFailoverPtr pool; /// Connection pool for each replica, contains nullptr for local replicas diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index da716d57f88..09a7bc41d33 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -178,6 +178,12 @@ void SelectStreamFactory::createForShard( return; } + if (shard_info.hasRemoteConnections()) + { + emplace_remote_stream(); + return; + } + const auto * replicated_storage = dynamic_cast(main_table_storage.get()); if (!replicated_storage) @@ -187,7 +193,7 @@ void SelectStreamFactory::createForShard( return; } - UInt64 max_allowed_delay = settings.max_replica_delay_for_distributed_queries; + const UInt64 max_allowed_delay = settings.max_replica_delay_for_distributed_queries; if (!max_allowed_delay) { From 220dc339377de870bf44e7777ca59cbeac1df2fb Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Sep 2023 18:46:24 +0000 Subject: [PATCH 04/93] Fast fix --- src/Interpreters/ClusterProxy/SelectStreamFactory.cpp | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 09a7bc41d33..7c6c93f5dde 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -146,7 +146,9 @@ void SelectStreamFactory::createForShard( return; }); - if (settings.prefer_localhost_replica && shard_info.isLocal()) + if (settings.prefer_localhost_replica && shard_info.isLocal() + && !context->canUseParallelReplicasOnInitiator()) // fast fix for parallel replicas over distributed with enabled perfer_localhost_replica + // basically, prefer_localhost_replica is disabled for now with parallel replicas over distributed { StoragePtr main_table_storage; @@ -178,12 +180,6 @@ void SelectStreamFactory::createForShard( return; } - if (shard_info.hasRemoteConnections()) - { - emplace_remote_stream(); - return; - } - const auto * replicated_storage = dynamic_cast(main_table_storage.get()); if (!replicated_storage) From 96657ba5a22816755c3e53ff8a9cdc93009971a5 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Sep 2023 18:49:17 +0000 Subject: [PATCH 05/93] Remove debug code --- src/Interpreters/Cluster.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index bb161702401..82c3d48bc05 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -642,7 +642,6 @@ void Cluster::initMisc() std::unique_ptr Cluster::getClusterWithReplicasAsShards(const Settings & settings, size_t max_replicas_from_shard) const { - LOG_DEBUG(&Poco::Logger::get(__FUNCTION__), "max_replicas_from_shard={}\n{}", max_replicas_from_shard, StackTrace().toString()); return std::unique_ptr{ new Cluster(ReplicasAsShardsTag{}, *this, settings, max_replicas_from_shard)}; } From 6c3ac83c07bfd03ab289b1ae8522cb71c221b8e2 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Sep 2023 20:08:05 +0000 Subject: [PATCH 06/93] Fix --- src/Interpreters/ClusterProxy/SelectStreamFactory.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index ada2c1679da..32625ec0ca4 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -147,8 +147,8 @@ void SelectStreamFactory::createForShard( }); if (settings.prefer_localhost_replica && shard_info.isLocal() - && !context->canUseParallelReplicasOnInitiator()) // fast fix for parallel replicas over distributed with enabled perfer_localhost_replica - // basically, prefer_localhost_replica is disabled for now with parallel replicas over distributed + // fast fix for parallel replicas over distributed with enabled perfer_localhost_replica, -> disable it for now + && (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS && settings.max_parallel_replicas > 1)) { StoragePtr main_table_storage; From 52dfaa54bf4cbba46ed45c6b3ec85f14259ee91c Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 8 Sep 2023 08:02:30 +0000 Subject: [PATCH 07/93] pass http retry timeout as ms Signed-off-by: Duc Canh Le --- src/IO/HTTPCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index add3e96c2c1..ef03acdde5d 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -321,7 +321,7 @@ namespace /// To avoid such a deadlock we unlock `lock` before entering `pool_ptr->second->get`. lock.unlock(); - auto retry_timeout = timeouts.connection_timeout.totalMicroseconds(); + auto retry_timeout = timeouts.connection_timeout.totalMilliseconds(); auto session = pool_ptr->second->get(retry_timeout); setTimeouts(*session, timeouts); From ffa82e9297c1d01f77793cc7b43aa4fb7bbec9c4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Sep 2023 16:58:13 +0200 Subject: [PATCH 08/93] Fix filtering parts with indexHint for non analyzer Signed-off-by: Azat Khuzhin --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 5 ++++- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 3 +++ tests/analyzer_tech_debt.txt | 1 + .../0_stateless/02880_indexHint__partition_id.reference | 9 +++++++++ .../0_stateless/02880_indexHint__partition_id.sql | 9 +++++++++ 5 files changed, 26 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02880_indexHint__partition_id.reference create mode 100644 tests/queries/0_stateless/02880_indexHint__partition_id.sql diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 4b95b74c3af..32802d5fa02 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1271,7 +1271,10 @@ static void buildIndexes( } /// TODO Support row_policy_filter and additional_filters - indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context); + if (settings.allow_experimental_analyzer) + indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context); + else + indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, query_info.query, context); indexes->use_skip_indexes = settings.use_skip_indexes; bool final = query_info.isFinal(); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 31aa2dbb61f..9c93ecb14c9 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -814,6 +814,9 @@ std::optional> MergeTreeDataSelectExecutor::filterPar ASTPtr expression_ast; auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */); + if (virtual_columns_block.rows() == 0) + return {}; + // Generate valid expressions for filtering VirtualColumnUtils::prepareFilterBlockWithQuery(query, context, virtual_columns_block, expression_ast); diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 4419190e12c..652ab0b99de 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -54,6 +54,7 @@ 01710_projection_additional_filters 01721_join_implicit_cast_long 01739_index_hint +02880_indexHint__partition_id 01747_join_view_filter_dictionary 01748_partition_id_pruning 01756_optimize_skip_unused_shards_rewrite_in diff --git a/tests/queries/0_stateless/02880_indexHint__partition_id.reference b/tests/queries/0_stateless/02880_indexHint__partition_id.reference new file mode 100644 index 00000000000..365e7b676c7 --- /dev/null +++ b/tests/queries/0_stateless/02880_indexHint__partition_id.reference @@ -0,0 +1,9 @@ +-- { echoOn } +select * from data prewhere indexHint(_partition_id = '1'); +1 +select count() from data prewhere indexHint(_partition_id = '1'); +1 +select * from data where indexHint(_partition_id = '1'); +1 +select count() from data where indexHint(_partition_id = '1'); +1 diff --git a/tests/queries/0_stateless/02880_indexHint__partition_id.sql b/tests/queries/0_stateless/02880_indexHint__partition_id.sql new file mode 100644 index 00000000000..d15b3f4ccea --- /dev/null +++ b/tests/queries/0_stateless/02880_indexHint__partition_id.sql @@ -0,0 +1,9 @@ +drop table if exists data; +create table data (part Int) engine=MergeTree() order by tuple() partition by part; +insert into data values (1)(2); + +-- { echoOn } +select * from data prewhere indexHint(_partition_id = '1'); +select count() from data prewhere indexHint(_partition_id = '1'); +select * from data where indexHint(_partition_id = '1'); +select count() from data where indexHint(_partition_id = '1'); From b239d409b1b323734205396943bf8e6282ea72e1 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 8 Sep 2023 13:28:09 -0300 Subject: [PATCH 09/93] Update query-complexity.md --- docs/en/operations/settings/query-complexity.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index 14d612be831..cf655e875ac 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -154,6 +154,13 @@ Result: Maximum query execution time in seconds. At this time, it is not checked for one of the sorting stages, or when merging and finalizing aggregate functions. +The `max_execution_time` parameter can be a bit tricky to understand. +It operates based on interpolation relative to the current query execution speed (this behaviour is controlled by [timeout_before_checking_execution_speed](#timeout-before-checking-execution-speed). +ClickHouse will interrupt a query if the projected execution time exceeds the specified max_execution_time. + +By default, the timeout_before_checking_execution_speed is set to 1 second. This means that after just one second of query execution, ClickHouse will begin estimating the total execution time. If, for example, `max_execution_time` is set to 3600 seconds (1 hour), ClickHouse will terminate the query if the estimated time exceeds this 3600-second limit. +If you set `timeout_before_checking_execution_speed `to 0, ClickHouse will use clock time as the basis for `max_execution_time`. + ## timeout_overflow_mode {#timeout-overflow-mode} What to do if the query is run longer than ‘max_execution_time’: ‘throw’ or ‘break’. By default, throw. From bb3c6ff68297d0789d1907d402558b2512cd94da Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 8 Sep 2023 13:30:13 -0300 Subject: [PATCH 10/93] Update query-complexity.md --- docs/en/operations/settings/query-complexity.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index cf655e875ac..a4c8f36b201 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -156,9 +156,9 @@ At this time, it is not checked for one of the sorting stages, or when merging a The `max_execution_time` parameter can be a bit tricky to understand. It operates based on interpolation relative to the current query execution speed (this behaviour is controlled by [timeout_before_checking_execution_speed](#timeout-before-checking-execution-speed). -ClickHouse will interrupt a query if the projected execution time exceeds the specified max_execution_time. - -By default, the timeout_before_checking_execution_speed is set to 1 second. This means that after just one second of query execution, ClickHouse will begin estimating the total execution time. If, for example, `max_execution_time` is set to 3600 seconds (1 hour), ClickHouse will terminate the query if the estimated time exceeds this 3600-second limit. +ClickHouse will interrupt a query if the projected execution time exceeds the specified `max_execution_time`. +By default, the timeout_before_checking_execution_speed is set to 1 second. This means that after just one second of query execution, ClickHouse will begin estimating the total execution time. +If, for example, `max_execution_time` is set to 3600 seconds (1 hour), ClickHouse will terminate the query if the estimated time exceeds this 3600-second limit. If you set `timeout_before_checking_execution_speed `to 0, ClickHouse will use clock time as the basis for `max_execution_time`. ## timeout_overflow_mode {#timeout-overflow-mode} From 9462b58616eff2518025d24bfba2cd6134cfd2ab Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 8 Sep 2023 13:32:24 -0300 Subject: [PATCH 11/93] Update query-complexity.md --- docs/en/operations/settings/query-complexity.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index a4c8f36b201..b65ecdcb6ab 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -155,7 +155,7 @@ Maximum query execution time in seconds. At this time, it is not checked for one of the sorting stages, or when merging and finalizing aggregate functions. The `max_execution_time` parameter can be a bit tricky to understand. -It operates based on interpolation relative to the current query execution speed (this behaviour is controlled by [timeout_before_checking_execution_speed](#timeout-before-checking-execution-speed). +It operates based on interpolation relative to the current query execution speed (this behaviour is controlled by [timeout_before_checking_execution_speed](#timeout-before-checking-execution-speed)). ClickHouse will interrupt a query if the projected execution time exceeds the specified `max_execution_time`. By default, the timeout_before_checking_execution_speed is set to 1 second. This means that after just one second of query execution, ClickHouse will begin estimating the total execution time. If, for example, `max_execution_time` is set to 3600 seconds (1 hour), ClickHouse will terminate the query if the estimated time exceeds this 3600-second limit. From 3a78ba6de4acd52ea22f6a9ea6757f7b740344a7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 9 Sep 2023 21:36:39 +0000 Subject: [PATCH 12/93] Fix, enable tests, make cluster_for_parallel_replicas empty by default --- src/Core/Settings.h | 2 +- .../ClusterProxy/SelectStreamFactory.cpp | 7 +++---- .../ClusterProxy/SelectStreamFactory.h | 3 ++- src/Interpreters/ClusterProxy/executeQuery.cpp | 14 ++++++++++---- .../test.py | 8 ++++++++ 5 files changed, 24 insertions(+), 10 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 620cc8fd67f..3c94c792ef4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -169,7 +169,7 @@ class IColumn; M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \ M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ \ - M(String, cluster_for_parallel_replicas, "default", "Cluster for a shard in which current server is located", 0) \ + M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 32625ec0ca4..2f598fda3a9 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -114,7 +114,8 @@ void SelectStreamFactory::createForShard( ContextPtr context, std::vector & local_plans, Shards & remote_shards, - UInt32 shard_count) + UInt32 shard_count, + bool parallel_replicas_enabled) { auto it = objects_by_shard.find(shard_info.shard_num); if (it != objects_by_shard.end()) @@ -146,9 +147,7 @@ void SelectStreamFactory::createForShard( return; }); - if (settings.prefer_localhost_replica && shard_info.isLocal() - // fast fix for parallel replicas over distributed with enabled perfer_localhost_replica, -> disable it for now - && (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS && settings.max_parallel_replicas > 1)) + if (settings.prefer_localhost_replica && shard_info.isLocal() && !parallel_replicas_enabled) { StoragePtr main_table_storage; diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index ca07fd5deda..a821730657d 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -78,7 +78,8 @@ public: ContextPtr context, std::vector & local_plans, Shards & remote_shards, - UInt32 shard_count); + UInt32 shard_count, + bool parallel_replicas_enabled); struct ShardPlans { diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 34be2636917..12d98f44ec8 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -178,8 +178,9 @@ void executeQuery( main_table, query_info.additional_filter_ast, log); new_context->increaseDistributedDepth(); - size_t shards = query_info.getCluster()->getShardCount(); - for (const auto & shard_info : query_info.getCluster()->getShardsInfo()) + ClusterPtr cluster = query_info.getCluster(); + const size_t shards = cluster->getShardCount(); + for (const auto & shard_info : cluster->getShardsInfo()) { ASTPtr query_ast_for_shard = query_ast->clone(); if (sharding_key_expr && query_info.optimized_cluster && settings.optimize_skip_unused_shards_rewrite_in && shards > 1) @@ -210,9 +211,14 @@ void executeQuery( } } + const auto & addresses = cluster->getShardsAddresses().at(shard_info.shard_num - 1); + const bool parallel_replicas_enabled = addresses.size() > 1 + && context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS && settings.max_parallel_replicas > 1; + stream_factory.createForShard(shard_info, query_ast_for_shard, main_table, table_func_ptr, - new_context, plans, remote_shards, static_cast(shards)); + new_context, plans, remote_shards, static_cast(shards), + parallel_replicas_enabled); } if (!remote_shards.empty()) @@ -236,7 +242,7 @@ void executeQuery( log, shards, query_info.storage_limits, - query_info.getCluster()->getName()); + not_optimized_cluster->getName()); read_from_remote->setStepDescription("Read from remote replica"); plan->addStep(std::move(read_from_remote)); diff --git a/tests/integration/test_parallel_replicas_over_distributed/test.py b/tests/integration/test_parallel_replicas_over_distributed/test.py index 50db95c90b2..5716a33aa09 100644 --- a/tests/integration/test_parallel_replicas_over_distributed/test.py +++ b/tests/integration/test_parallel_replicas_over_distributed/test.py @@ -106,10 +106,18 @@ def create_tables(cluster, table_name): pytest.param("test_single_shard_multiple_replicas", 3, 0), pytest.param("test_single_shard_multiple_replicas", 4, 0), pytest.param("test_single_shard_multiple_replicas", 10, 0), + pytest.param("test_single_shard_multiple_replicas", 2, 1), + pytest.param("test_single_shard_multiple_replicas", 3, 1), + pytest.param("test_single_shard_multiple_replicas", 4, 1), + pytest.param("test_single_shard_multiple_replicas", 10, 1), pytest.param("test_multiple_shards_multiple_replicas", 2, 0), pytest.param("test_multiple_shards_multiple_replicas", 3, 0), pytest.param("test_multiple_shards_multiple_replicas", 4, 0), pytest.param("test_multiple_shards_multiple_replicas", 10, 0), + pytest.param("test_multiple_shards_multiple_replicas", 2, 1), + pytest.param("test_multiple_shards_multiple_replicas", 3, 1), + pytest.param("test_multiple_shards_multiple_replicas", 4, 1), + pytest.param("test_multiple_shards_multiple_replicas", 10, 1), ], ) def test_parallel_replicas_over_distributed( From fac0b2f62dc95f776ee6ad174c3070005ae47009 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 10 Sep 2023 19:05:17 +0000 Subject: [PATCH 13/93] Adapt code to inconsistency between shard_info and shard addresses --- src/Interpreters/ClusterProxy/executeQuery.cpp | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 12d98f44ec8..401eea7c39a 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -211,9 +211,19 @@ void executeQuery( } } - const auto & addresses = cluster->getShardsAddresses().at(shard_info.shard_num - 1); - const bool parallel_replicas_enabled = addresses.size() > 1 - && context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS && settings.max_parallel_replicas > 1; + bool parallel_replicas_enabled = false; + if (shard_info.shard_num > 0 && shard_info.shard_num <= cluster->getShardsAddresses().size()) + { + const auto & addresses = cluster->getShardsAddresses().at(shard_info.shard_num - 1); + parallel_replicas_enabled = addresses.size() > 1 + && context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS && settings.max_parallel_replicas > 1; + } + else + { + chassert(shard_info.shard_num > 0); + + // FIXME or code: when can it happened (shard_num bigger than shard's addresses)? looks inconsistent + } stream_factory.createForShard(shard_info, query_ast_for_shard, main_table, table_func_ptr, From 7d630b6b1bee7839d295eb5a548e17e94012fb93 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 10 Sep 2023 20:38:17 +0000 Subject: [PATCH 14/93] Added comments --- src/Interpreters/ClusterProxy/SelectStreamFactory.cpp | 3 +++ src/Interpreters/ClusterProxy/executeQuery.cpp | 2 ++ 2 files changed, 5 insertions(+) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 2f598fda3a9..3935028f27c 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -147,6 +147,9 @@ void SelectStreamFactory::createForShard( return; }); + // prefer_localhost_replica is not effective in case of parallel replicas + // (1) prefer_localhost_replica is about choosing one replica on a shard + // (2) parallel replica coordinator has own logic to choose replicas to read from if (settings.prefer_localhost_replica && shard_info.isLocal() && !parallel_replicas_enabled) { StoragePtr main_table_storage; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 401eea7c39a..bd8f5cea7f3 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -211,6 +211,8 @@ void executeQuery( } } + // decide for each shard if parallel reading from replicas should be enabled + // according to settings and number of replicas declared per shard bool parallel_replicas_enabled = false; if (shard_info.shard_num > 0 && shard_info.shard_num <= cluster->getShardsAddresses().size()) { From 918bd814bd4a274675dc42b12231795a5f582e31 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 11 Sep 2023 12:19:07 +0000 Subject: [PATCH 15/93] Add canUseParallelReplicas() for convenience --- src/Interpreters/ClusterProxy/executeQuery.cpp | 3 +-- src/Interpreters/Context.cpp | 12 ++++++++---- src/Interpreters/Context.h | 1 + 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index bd8f5cea7f3..4395f8373b2 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -217,8 +217,7 @@ void executeQuery( if (shard_info.shard_num > 0 && shard_info.shard_num <= cluster->getShardsAddresses().size()) { const auto & addresses = cluster->getShardsAddresses().at(shard_info.shard_num - 1); - parallel_replicas_enabled = addresses.size() > 1 - && context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS && settings.max_parallel_replicas > 1; + parallel_replicas_enabled = addresses.size() > 1 && context->canUseParallelReplicas(); } else { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ee7efdfeb1b..7834f5893d3 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4626,18 +4626,22 @@ Context::ParallelReplicasMode Context::getParallelReplicasMode() const return SAMPLE_KEY; } +bool Context::canUseParallelReplicas() const +{ + const auto & settings_ref = getSettingsRef(); + return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS && settings_ref.max_parallel_replicas > 1; +} + bool Context::canUseParallelReplicasOnInitiator() const { const auto & settings_ref = getSettingsRef(); - return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS && settings_ref.max_parallel_replicas > 1 - && !getClientInfo().collaborate_with_initiator; + return canUseParallelReplicas() && !getClientInfo().collaborate_with_initiator; } bool Context::canUseParallelReplicasOnFollower() const { const auto & settings_ref = getSettingsRef(); - return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS && settings_ref.max_parallel_replicas > 1 - && getClientInfo().collaborate_with_initiator; + return canUseParallelReplicas() && getClientInfo().collaborate_with_initiator; } void Context::setPreparedSetsCache(const PreparedSetsCachePtr & cache) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b4a5b3d8c85..e2b9c930249 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1174,6 +1174,7 @@ public: WriteSettings getWriteSettings() const; /** There are multiple conditions that have to be met to be able to use parallel replicas */ + bool canUseParallelReplicas() const; bool canUseParallelReplicasOnInitiator() const; bool canUseParallelReplicasOnFollower() const; From 1785e1c00c0987f8f1cf5ccff23db04fe18e7009 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 11 Sep 2023 12:30:55 +0000 Subject: [PATCH 16/93] Fix build: remove unused variables --- src/Interpreters/Context.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7834f5893d3..526a644ce6f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4634,13 +4634,11 @@ bool Context::canUseParallelReplicas() const bool Context::canUseParallelReplicasOnInitiator() const { - const auto & settings_ref = getSettingsRef(); return canUseParallelReplicas() && !getClientInfo().collaborate_with_initiator; } bool Context::canUseParallelReplicasOnFollower() const { - const auto & settings_ref = getSettingsRef(); return canUseParallelReplicas() && getClientInfo().collaborate_with_initiator; } From ca57eb95466b3532bae295fda9e42004d9e88301 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Thu, 14 Sep 2023 15:55:37 +0200 Subject: [PATCH 17/93] Fix addData function --- src/Functions/FunctionsOpDate.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/FunctionsOpDate.cpp b/src/Functions/FunctionsOpDate.cpp index 0d8ca2b58cc..dcbc1fc1256 100644 --- a/src/Functions/FunctionsOpDate.cpp +++ b/src/Functions/FunctionsOpDate.cpp @@ -53,7 +53,6 @@ public: } bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 2}; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { From 7643e216bead8c98d742427869cd121655cbdd28 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Thu, 14 Sep 2023 15:58:00 +0200 Subject: [PATCH 18/93] Better test --- tests/queries/0_stateless/02834_add_sub_date_functions.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02834_add_sub_date_functions.sql b/tests/queries/0_stateless/02834_add_sub_date_functions.sql index 44d9bb8a2aa..9c77ba7e378 100644 --- a/tests/queries/0_stateless/02834_add_sub_date_functions.sql +++ b/tests/queries/0_stateless/02834_add_sub_date_functions.sql @@ -1,6 +1,6 @@ SET session_timezone = 'UTC'; -SELECT ADDDATE('2022-05-07'::Date, INTERVAL 5 MINUTE); +SELECT ADDDATE(materialize('2022-05-07'::Date), INTERVAL 5 MINUTE); SELECT addDate('2022-05-07'::Date, INTERVAL 5 MINUTE); SELECT addDate('2022-05-07'::Date32, INTERVAL 5 MINUTE); @@ -14,7 +14,7 @@ SELECT addDate('1234', INTERVAL 5 MINUTE); -- { serverError ILLEGAL_TYPE_OF_ARG SELECT '---'; -SELECT SUBDATE('2022-05-07'::Date, INTERVAL 5 MINUTE); +SELECT SUBDATE(materialize('2022-05-07'::Date), INTERVAL 5 MINUTE); SELECT subDate('2022-05-07'::Date, INTERVAL 5 MINUTE); SELECT subDate('2022-05-07'::Date32, INTERVAL 5 MINUTE); From d9e15c00c99c45bdba546173bbcfaf1698196404 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 14 Sep 2023 19:45:07 +0200 Subject: [PATCH 19/93] limit the delay before next try in S3 --- src/Backups/BackupIO_S3.cpp | 1 + src/Coordination/KeeperSnapshotManagerS3.cpp | 3 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 1 + src/IO/S3/Client.cpp | 51 +++++++++----------- src/IO/S3/Client.h | 17 +++---- src/IO/S3/Credentials.cpp | 3 ++ src/IO/S3/PocoHTTPClient.cpp | 2 + src/IO/S3/PocoHTTPClient.h | 2 + src/IO/S3/tests/gtest_aws_s3_client.cpp | 11 +---- src/IO/tests/gtest_writebuffer_s3.cpp | 1 + src/Storages/StorageS3.cpp | 4 +- 11 files changed, 44 insertions(+), 52 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 2482679fa4e..ef820784bdf 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -49,6 +49,7 @@ namespace settings.auth_settings.region, context->getRemoteHostFilter(), static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), + static_cast(context->getGlobalContext()->getSettingsRef().s3_retry_attempts), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, /* for_disk_s3 = */ false, settings.request_settings.get_request_throttler, settings.request_settings.put_request_throttler, s3_uri.uri.getScheme()); diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index e44b9f376c7..302e05c8418 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -80,6 +80,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo auto headers = auth_settings.headers; static constexpr size_t s3_max_redirects = 10; + static constexpr size_t s3_retry_attempts = 10; static constexpr bool enable_s3_requests_logging = false; if (!new_uri.key.empty()) @@ -90,7 +91,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( auth_settings.region, - RemoteHostFilter(), s3_max_redirects, + RemoteHostFilter(), s3_max_redirects, s3_retry_attempts, enable_s3_requests_logging, /* for_disk_s3 = */ false, /* get_request_throttler = */ {}, /* put_request_throttler = */ {}, new_uri.uri.getScheme()); diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 06e79193c3e..3e38d0a7912 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -52,6 +52,7 @@ std::unique_ptr getClient( config.getString(config_prefix + ".region", ""), context->getRemoteHostFilter(), static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), + static_cast(context->getGlobalContext()->getSettingsRef().s3_retry_attempts), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, /* for_disk_s3 = */ true, settings.request_settings.get_request_throttler, diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 4b6968f363a..88578b16cc1 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -49,11 +49,12 @@ namespace ErrorCodes namespace S3 { -Client::RetryStrategy::RetryStrategy(std::shared_ptr wrapped_strategy_) - : wrapped_strategy(std::move(wrapped_strategy_)) +Client::RetryStrategy::RetryStrategy(uint32_t maxRetries_, uint32_t scaleFactor_, uint32_t maxDelayMs_) + : maxRetries(maxRetries_) + , scaleFactor(scaleFactor_) + , maxDelayMs(maxDelayMs_) { - if (!wrapped_strategy) - wrapped_strategy = Aws::Client::InitRetryStrategy(); + chassert(maxDelayMs <= uint64_t(scaleFactor) * (1ul << 31l)); } /// NOLINTNEXTLINE(google-runtime-int) @@ -62,39 +63,28 @@ bool Client::RetryStrategy::ShouldRetry(const Aws::Client::AWSErrorShouldRetry(error, attemptedRetries); + if (attemptedRetries >= maxRetries) + return false; + + return error.ShouldRetry(); } /// NOLINTNEXTLINE(google-runtime-int) -long Client::RetryStrategy::CalculateDelayBeforeNextRetry(const Aws::Client::AWSError& error, long attemptedRetries) const +long Client::RetryStrategy::CalculateDelayBeforeNextRetry(const Aws::Client::AWSError&, long attemptedRetries) const { - return wrapped_strategy->CalculateDelayBeforeNextRetry(error, attemptedRetries); + if (attemptedRetries == 0) + { + return 0; + } + + uint64_t backoffLimitedPow = 1ul << std::min(attemptedRetries, 31l); + return std::min(scaleFactor * backoffLimitedPow, maxDelayMs); } /// NOLINTNEXTLINE(google-runtime-int) long Client::RetryStrategy::GetMaxAttempts() const { - return wrapped_strategy->GetMaxAttempts(); -} - -void Client::RetryStrategy::GetSendToken() -{ - return wrapped_strategy->GetSendToken(); -} - -bool Client::RetryStrategy::HasSendToken() -{ - return wrapped_strategy->HasSendToken(); -} - -void Client::RetryStrategy::RequestBookkeeping(const Aws::Client::HttpResponseOutcome& httpResponseOutcome) -{ - return wrapped_strategy->RequestBookkeeping(httpResponseOutcome); -} - -void Client::RetryStrategy::RequestBookkeeping(const Aws::Client::HttpResponseOutcome& httpResponseOutcome, const Aws::Client::AWSError& lastError) -{ - return wrapped_strategy->RequestBookkeeping(httpResponseOutcome, lastError); + return maxRetries; } namespace @@ -846,7 +836,8 @@ std::unique_ptr ClientFactory::create( // NOLINT std::move(credentials), credentials_configuration); - client_configuration.retryStrategy = std::make_shared(std::move(client_configuration.retryStrategy)); + client_configuration.retryStrategy = std::make_shared(client_configuration.s3_retry_attempts); + return Client::create( client_configuration.s3_max_redirects, std::move(sse_kms_config), @@ -861,6 +852,7 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT const String & force_region, const RemoteHostFilter & remote_host_filter, unsigned int s3_max_redirects, + unsigned int s3_retry_attempts, bool enable_s3_requests_logging, bool for_disk_s3, const ThrottlerPtr & get_request_throttler, @@ -879,6 +871,7 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT force_region, remote_host_filter, s3_max_redirects, + s3_retry_attempts, enable_s3_requests_logging, for_disk_s3, get_request_throttler, diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index a2be7ff5566..48310bc21af 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -152,16 +152,16 @@ public: Aws::Auth::AWSCredentials getCredentials() const; - /// Decorator for RetryStrategy needed for this client to work correctly. /// We want to manually handle permanent moves (status code 301) because: /// - redirect location is written in XML format inside the response body something that doesn't exist for HEAD /// requests so we need to manually find the correct location /// - we want to cache the new location to decrease number of roundtrips for future requests - /// This decorator doesn't retry if 301 is detected and fallbacks to the inner retry strategy otherwise. + /// Other retries are processed with exponential backoff timeout + /// which is limited and rundomly spread class RetryStrategy : public Aws::Client::RetryStrategy { public: - explicit RetryStrategy(std::shared_ptr wrapped_strategy_); + RetryStrategy(uint32_t maxRetries_ = 10, uint32_t scaleFactor_ = 25, uint32_t maxDelayMs_ = 90000); /// NOLINTNEXTLINE(google-runtime-int) bool ShouldRetry(const Aws::Client::AWSError& error, long attemptedRetries) const override; @@ -172,14 +172,10 @@ public: /// NOLINTNEXTLINE(google-runtime-int) long GetMaxAttempts() const override; - void GetSendToken() override; - - bool HasSendToken() override; - - void RequestBookkeeping(const Aws::Client::HttpResponseOutcome& httpResponseOutcome) override; - void RequestBookkeeping(const Aws::Client::HttpResponseOutcome& httpResponseOutcome, const Aws::Client::AWSError& lastError) override; private: - std::shared_ptr wrapped_strategy; + uint32_t maxRetries; + uint32_t scaleFactor; + uint32_t maxDelayMs; }; /// SSE-KMS headers MUST be signed, so they need to be added before the SDK signs the message @@ -311,6 +307,7 @@ public: const String & force_region, const RemoteHostFilter & remote_host_filter, unsigned int s3_max_redirects, + unsigned int s3_retry_attempts, bool enable_s3_requests_logging, bool for_disk_s3, const ThrottlerPtr & get_request_throttler, diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 6ffd6fde43e..e02ab628e62 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -623,6 +623,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, + configuration.s3_retry_attempts, configuration.enable_s3_requests_logging, configuration.for_disk_s3, configuration.get_request_throttler, @@ -637,6 +638,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, + configuration.s3_retry_attempts, configuration.enable_s3_requests_logging, configuration.for_disk_s3, configuration.get_request_throttler, @@ -679,6 +681,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, + configuration.s3_retry_attempts, configuration.enable_s3_requests_logging, configuration.for_disk_s3, configuration.get_request_throttler, diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 90327d4dc2e..aced90d2d57 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -89,6 +89,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( const String & force_region_, const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, + unsigned int s3_retry_attempts_, bool enable_s3_requests_logging_, bool for_disk_s3_, const ThrottlerPtr & get_request_throttler_, @@ -98,6 +99,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( , force_region(force_region_) , remote_host_filter(remote_host_filter_) , s3_max_redirects(s3_max_redirects_) + , s3_retry_attempts(s3_retry_attempts_) , enable_s3_requests_logging(enable_s3_requests_logging_) , for_disk_s3(for_disk_s3_) , get_request_throttler(get_request_throttler_) diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index d1ce148a707..30220e3e284 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -41,6 +41,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration String force_region; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; + unsigned int s3_retry_attempts; bool enable_s3_requests_logging; bool for_disk_s3; ThrottlerPtr get_request_throttler; @@ -64,6 +65,7 @@ private: const String & force_region_, const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, + unsigned int s3_retry_attempts, bool enable_s3_requests_logging_, bool for_disk_s3_, const ThrottlerPtr & get_request_throttler_, diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 6d589bcedd5..c42f14e9a53 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -40,14 +40,6 @@ [[maybe_unused]] static Poco::Util::ServerApplication app; -class NoRetryStrategy : public Aws::Client::StandardRetryStrategy -{ - bool ShouldRetry(const Aws::Client::AWSError &, long /* NOLINT */) const override { return false; } - -public: - ~NoRetryStrategy() override = default; -}; - String getSSEAndSignedHeaders(const Poco::Net::MessageHeader & message_header) { String content; @@ -123,6 +115,7 @@ void testServerSideEncryption( DB::RemoteHostFilter remote_host_filter; unsigned int s3_max_redirects = 100; + unsigned int s3_retry_attempts = 0; DB::S3::URI uri(http.getUrl() + "/IOTestAwsS3ClientAppendExtraHeaders/test.txt"); String access_key_id = "ACCESS_KEY_ID"; String secret_access_key = "SECRET_ACCESS_KEY"; @@ -132,6 +125,7 @@ void testServerSideEncryption( region, remote_host_filter, s3_max_redirects, + s3_retry_attempts, enable_s3_requests_logging, /* for_disk_s3 = */ false, /* get_request_throttler = */ {}, @@ -140,7 +134,6 @@ void testServerSideEncryption( ); client_configuration.endpointOverride = uri.endpoint; - client_configuration.retryStrategy = std::make_shared(); DB::HTTPHeaderEntries headers; bool use_environment_credentials = false; diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index d14893c4f3e..21bdd9a6f26 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -228,6 +228,7 @@ struct Client : DB::S3::Client "some-region", remote_host_filter, /* s3_max_redirects = */ 100, + /* s3_retry_attempts = */ 0, /* enable_s3_requests_logging = */ true, /* for_disk_s3 = */ false, /* get_request_throttler = */ {}, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 341d8b3f768..15ec32bfd3f 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1204,6 +1204,7 @@ void StorageS3::Configuration::connect(ContextPtr context) auth_settings.region, context->getRemoteHostFilter(), static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), + static_cast(context->getGlobalContext()->getSettingsRef().s3_retry_attempts), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, /* for_disk_s3 = */ false, request_settings.get_request_throttler, @@ -1218,9 +1219,6 @@ void StorageS3::Configuration::connect(ContextPtr context) client_configuration.requestTimeoutMs = request_settings.request_timeout_ms; - client_configuration.retryStrategy - = std::make_shared(request_settings.retry_attempts); - auto credentials = Aws::Auth::AWSCredentials(auth_settings.access_key_id, auth_settings.secret_access_key); client = S3::ClientFactory::instance().create( client_configuration, From c422a8f0dc14269041aba360ea0ac645e2a635e1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 13 Sep 2023 13:41:02 +0000 Subject: [PATCH 20/93] Cosmetics --- docs/en/sql-reference/data-types/array.md | 2 +- .../MergeTree/MergeTreeIndexAnnoy.cpp | 26 +++++++++---------- .../MergeTree/MergeTreeIndexUSearch.cpp | 24 ++++++++--------- 3 files changed, 26 insertions(+), 26 deletions(-) diff --git a/docs/en/sql-reference/data-types/array.md b/docs/en/sql-reference/data-types/array.md index 20ce7d2ed52..0ee7c8de93c 100644 --- a/docs/en/sql-reference/data-types/array.md +++ b/docs/en/sql-reference/data-types/array.md @@ -4,7 +4,7 @@ sidebar_position: 52 sidebar_label: Array(T) --- -# Array(t) +# Array(T) An array of `T`-type items, with the starting array index as 1. `T` can be any data type, including an array. diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 3ad4f81716e..15830513162 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -154,36 +154,36 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t if (const auto & column_array = typeid_cast(column_cut.get())) { - const auto & data = column_array->getData(); - const auto & array = typeid_cast(data).getData(); + const auto & column_array_data = column_array->getData(); + const auto & column_arary_data_float_data = typeid_cast(column_array_data).getData(); - if (array.empty()) + if (column_arary_data_float_data.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read); - const auto & offsets = column_array->getOffsets(); - const size_t num_rows = offsets.size(); + const auto & column_array_offsets = column_array->getOffsets(); + const size_t num_rows = column_array_offsets.size(); /// Check all sizes are the same - size_t size = offsets[0]; + size_t dimension = column_array_offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) - if (offsets[i + 1] - offsets[i] != size) + if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension) throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); if (!index) - index = std::make_shared>(size); + index = std::make_shared>(dimension); /// Add all rows of block - index->add_item(index->get_n_items(), array.data()); + index->add_item(index->get_n_items(), column_arary_data_float_data.data()); for (size_t current_row = 1; current_row < num_rows; ++current_row) - index->add_item(index->get_n_items(), &array[offsets[current_row - 1]]); + index->add_item(index->get_n_items(), &column_arary_data_float_data[column_array_offsets[current_row - 1]]); } else if (const auto & column_tuple = typeid_cast(column_cut.get())) { - const auto & columns = column_tuple->getColumns(); + const auto & column_tuple_columns = column_tuple->getColumns(); /// TODO check if calling index->add_item() directly on the block's tuples is faster than materializing everything - std::vector> data{column_tuple->size(), std::vector()}; - for (const auto & column : columns) + std::vector> data(column_tuple->size(), std::vector()); + for (const auto & column : column_tuple_columns) { const auto & pod_array = typeid_cast(column.get())->getData(); for (size_t i = 0; i < pod_array.size(); ++i) diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index 1ab85e6bbaf..de556eb7e07 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -173,23 +173,23 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t if (const auto & column_array = typeid_cast(column_cut.get())) { - const auto & data = column_array->getData(); - const auto & array = typeid_cast(data).getData(); + const auto & column_array_data = column_array->getData(); + const auto & column_array_data_float_data = typeid_cast(column_array_data).getData(); - if (array.empty()) + if (column_array_data_float_data.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read); - const auto & offsets = column_array->getOffsets(); - const size_t num_rows = offsets.size(); + const auto & column_array_offsets = column_array->getOffsets(); + const size_t num_rows = column_array_offsets.size(); /// Check all sizes are the same - size_t size = offsets[0]; + size_t dimension = column_array_offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) - if (offsets[i + 1] - offsets[i] != size) + if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension) throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); if (!index) - index = std::make_shared>(size, scalar_kind); + index = std::make_shared>(dimension, scalar_kind); /// Add all rows of block if (!index->reserve(unum::usearch::ceil2(index->size() + num_rows))) @@ -197,7 +197,7 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t for (size_t current_row = 0; current_row < num_rows; ++current_row) { - auto rc = index->add(static_cast(index->size()), &array[offsets[current_row - 1]]); + auto rc = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[current_row - 1]]); if (!rc) throw Exception(ErrorCodes::INCORRECT_DATA, rc.error.release()); @@ -208,9 +208,9 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t } else if (const auto & column_tuple = typeid_cast(column_cut.get())) { - const auto & columns = column_tuple->getColumns(); - std::vector> data{column_tuple->size(), std::vector()}; - for (const auto & column : columns) + const auto & column_tuple_columns = column_tuple->getColumns(); + std::vector> data(column_tuple->size(), std::vector()); + for (const auto & column : column_tuple_columns) { const auto & pod_array = typeid_cast(column.get())->getData(); for (size_t i = 0; i < pod_array.size(); ++i) From 945179be46bcc1b07741d180a0fdaa64396994ef Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 13 Sep 2023 14:23:09 +0000 Subject: [PATCH 21/93] Annoy: Fix LOGICAL_ERROR with default values #52258 --- .../table-engines/mergetree-family/annindexes.md | 8 +++++--- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 15 +++++++++++---- src/Storages/MergeTree/MergeTreeIndexUSearch.cpp | 16 ++++++++++++---- .../0_stateless/02354_annoy_index.reference | 1 + tests/queries/0_stateless/02354_annoy_index.sql | 12 ++++++++++++ .../0_stateless/02354_usearch_index.reference | 1 + .../queries/0_stateless/02354_usearch_index.sql | 14 ++++++++++++++ 7 files changed, 56 insertions(+), 11 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 8996133f667..d6ff7f23bb4 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -203,9 +203,10 @@ Parameter `NumTrees` is the number of trees which the algorithm creates (default more accurate search results but slower index creation / query times (approximately linearly) as well as larger index sizes. :::note -Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use -[CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 -CHECK length(vectors) = 256`. +Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays must have same length. To avoid +errors, you can use a [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints), for example, `CONSTRAINT +constraint_name_1 CHECK length(vectors) = 256`. Also, unspecified `Array` values in INSERT statements (i.e. default values) are not +supported. ::: Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. Larger @@ -223,6 +224,7 @@ SETTINGS annoy_index_search_k_nodes=100; The Annoy index currently does not work with per-table, non-default `index_granularity` settings (see [here](https://github.com/ClickHouse/ClickHouse/pull/51325#issuecomment-1605920475)). If necessary, the value must be changed in config.xml. ::: + ## USearch {#usearch} This type of ANN index is based on the [the USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 15830513162..f00f11359e1 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -157,18 +157,25 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t const auto & column_array_data = column_array->getData(); const auto & column_arary_data_float_data = typeid_cast(column_array_data).getData(); - if (column_arary_data_float_data.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read); - const auto & column_array_offsets = column_array->getOffsets(); const size_t num_rows = column_array_offsets.size(); + /// The index dimension is inferred from the inserted arrays (array cardinality). If no value was specified in the INSERT statement + /// for the annoy-indexed column (i.e. default value), we have a problem. Reject such values. + if (column_array_offsets.empty() || column_array_offsets[0] == 0) + /// (The if condition is a bit weird but I have seen either with default values) + throw Exception(ErrorCodes::INCORRECT_DATA, "Tried to insert {} rows into Annoy index but there were no values to insert. Likely, the INSERT used default values - these are not supported for Annoy.", rows_read); + /// Check all sizes are the same size_t dimension = column_array_offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension) throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + /// Also check that previously inserted blocks have the same size as this block + if (index && index->getDimensions() != dimension) + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + if (!index) index = std::make_shared>(dimension); @@ -363,7 +370,7 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) { throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Annoy indexes can only be created on columns of type Array(Float32) and Tuple(Float32)"); + "Annoy indexes can only be created on columns of type Array(Float32) and Tuple(Float32[, Float32[, ...]])"); }; DataTypePtr data_type = index.sample_block.getDataTypes()[0]; diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index de556eb7e07..9531b9188bf 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -176,18 +176,25 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t const auto & column_array_data = column_array->getData(); const auto & column_array_data_float_data = typeid_cast(column_array_data).getData(); - if (column_array_data_float_data.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read); - const auto & column_array_offsets = column_array->getOffsets(); const size_t num_rows = column_array_offsets.size(); + /// The index dimension is inferred from the inserted arrays (array cardinality). If no value was specified in the INSERT statement + /// for the usearch-indexed column (i.e. default value), we have a problem. Reject such values. + if (column_array_offsets.empty() || column_array_offsets[0] == 0) + /// (The if condition is a bit weird but I have seen either with default values) + throw Exception(ErrorCodes::INCORRECT_DATA, "Tried to insert {} rows into usearch index but there were no values to insert. Likely, the INSERT used default values - these are not supported for Annoy.", rows_read); + /// Check all sizes are the same size_t dimension = column_array_offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension) throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + /// Also check that previously inserted blocks have the same size as this block + if (index && index->getDimensions() != dimension) + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + if (!index) index = std::make_shared>(dimension, scalar_kind); @@ -413,7 +420,8 @@ void usearchIndexValidator(const IndexDescription & index, bool /* attach */) auto throw_unsupported_underlying_column_exception = []() { throw Exception( - ErrorCodes::ILLEGAL_COLUMN, "USearch indexes can only be created on columns of type Array(Float32) and Tuple(Float32)"); + ErrorCodes::ILLEGAL_COLUMN, + "USearch can only be created on columns of type Array(Float32) and Tuple(Float32[, Float32[, ...]])"); }; DataTypePtr data_type = index.sample_block.getDataTypes()[0]; diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference index a0ffb1e1f7f..81f2ff8aa59 100644 --- a/tests/queries/0_stateless/02354_annoy_index.reference +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -147,3 +147,4 @@ Expression (Projection) 9000 [9000,0,0,0] 1 (1,0,0,0) 9000 (9000,0,0,0) +--- Bugs --- diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index eab7a62c5f0..67ef64cc301 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -281,3 +281,15 @@ ORDER BY L2Distance(vector, (9000.0, 0.0, 0.0, 0.0)) LIMIT 1; DROP TABLE tab; + +SELECT '--- Bugs ---'; + +-- Arrays with default values are rejected, issue #52258 +CREATE TABLE tab (`uuid` String, `vector` Array(Float32), `version` UInt32, INDEX idx vector TYPE annoy()) ENGINE = MergeTree() ORDER BY (uuid); +INSERT INTO tab (uuid, version) VALUES ('1', 3); -- { serverError INCORRECT_DATA } +DROP TABLE tab; + +-- Tuples with default value work +CREATE TABLE tab (`uuid` String, `vector` Tuple(Float32, Float32), `version` UInt32, INDEX idx vector TYPE annoy()) ENGINE = MergeTree() ORDER BY (uuid); +INSERT INTO tab (uuid, version) VALUES ('1', 3); -- works fine +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_usearch_index.reference b/tests/queries/0_stateless/02354_usearch_index.reference index 893a092a386..c2791e99a54 100644 --- a/tests/queries/0_stateless/02354_usearch_index.reference +++ b/tests/queries/0_stateless/02354_usearch_index.reference @@ -150,3 +150,4 @@ Expression (Projection) 1 [0,0,10] 2 [0,0,10.5] 3 [0,0,9.5] +--- Bugs --- diff --git a/tests/queries/0_stateless/02354_usearch_index.sql b/tests/queries/0_stateless/02354_usearch_index.sql index e534c91b615..fc2954d6c5d 100644 --- a/tests/queries/0_stateless/02354_usearch_index.sql +++ b/tests/queries/0_stateless/02354_usearch_index.sql @@ -274,3 +274,17 @@ SELECT * FROM tab WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < 1.0 LIMIT 3; + +DROP TABLE tab; + +SELECT '--- Bugs ---'; + +-- Arrays with default values are rejected, issue #52258 +CREATE TABLE tab (`uuid` String, `vector` Array(Float32), `version` UInt32, INDEX idx vector TYPE usearch()) ENGINE = MergeTree() ORDER BY (uuid); +INSERT INTO tab (uuid, version) VALUES ('1', 3); -- { serverError INCORRECT_DATA } +DROP TABLE tab; + +-- Tuples with default value work +CREATE TABLE tab (`uuid` String, `vector` Tuple(Float32, Float32), `version` UInt32, INDEX idx vector TYPE usearch()) ENGINE = MergeTree() ORDER BY (uuid); +INSERT INTO tab (uuid, version) VALUES ('1', 3); -- works fine +DROP TABLE tab; From 0edfa22181cedb1ac50ffdd7f5072831a04fa1f5 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 15 Sep 2023 13:13:14 +0200 Subject: [PATCH 22/93] attempts count is retries plus 1 --- src/IO/S3/Client.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 88578b16cc1..ceae33aee25 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -84,7 +84,7 @@ long Client::RetryStrategy::CalculateDelayBeforeNextRetry(const Aws::Client::AWS /// NOLINTNEXTLINE(google-runtime-int) long Client::RetryStrategy::GetMaxAttempts() const { - return maxRetries; + return maxRetries + 1; } namespace @@ -558,9 +558,10 @@ Client::doRequestWithRetryNetworkErrors(const RequestType & request, RequestFn r auto with_retries = [this, request_fn_ = std::move(request_fn)] (const RequestType & request_) { chassert(client_configuration.retryStrategy); - const Int64 max_attempts = client_configuration.retryStrategy->GetMaxAttempts(); + const long max_attempts = client_configuration.retryStrategy->GetMaxAttempts(); + chassert(max_attempts > 0); std::exception_ptr last_exception = nullptr; - for (Int64 attempt_no = 0; attempt_no < max_attempts; ++attempt_no) + for (long attempt_no = 0; attempt_no < max_attempts; ++attempt_no) { try { From 68c6bd882663ed6070d0cbbb0f6c62b5ee0e7ddc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 15 Sep 2023 12:04:22 +0000 Subject: [PATCH 23/93] Make the failed message single line to fix result parsing --- src/Parsers/tests/gtest_Parser.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 18e91c533e0..57588d1d5e5 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -39,7 +39,9 @@ std::ostream & operator<<(std::ostream & ostr, const std::shared_ptr pa std::ostream & operator<<(std::ostream & ostr, const ParserTestCase & test_case) { - return ostr << "ParserTestCase input: " << test_case.input_text; + std::string input_text{test_case.input_text}; + std::replace(input_text.begin(), input_text.end(),'\n', ' '); + return ostr << "ParserTestCase input: " << input_text; } class ParserTest : public ::testing::TestWithParam, ParserTestCase>> From 358f22d18a57ef85481742c45cb67c6c4515ac87 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 15 Sep 2023 12:31:30 +0000 Subject: [PATCH 24/93] Update PRQL unit tests to latest PRQL version --- src/Parsers/tests/gtest_Parser.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 57588d1d5e5..a20d6b2f111 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -496,11 +496,11 @@ INSTANTIATE_TEST_SUITE_P( ::testing::Values(std::make_shared(kDummyMaxQuerySize, kDummyMaxParserDepth)), ::testing::ValuesIn(std::initializer_list{ { - "from albums\ngroup [author_id] (\n aggregate [first_pushlied = min published]\n)\njoin a=author side:left [==author_id]\njoin p=purchases side:right [==author_id]\ngroup [a.id, p.purchase_id] (\n aggregate [avg_sell = min first_pushlied]\n)", - "WITH table_1 AS\n (\n SELECT\n MIN(published) AS _expr_0,\n author_id\n FROM albums\n GROUP BY author_id\n )\nSELECT\n a.id,\n p.purchase_id,\n MIN(table_0._expr_0) AS avg_sell\nFROM table_1 AS table_0\nLEFT JOIN author AS a ON table_0.author_id = a.author_id\nRIGHT JOIN purchases AS p ON table_0.author_id = p.author_id\nGROUP BY\n a.id,\n p.purchase_id", + "from albums\ngroup {author_id} (\n aggregate {first_published = min published}\n)\njoin a=author side:left (==author_id)\njoin p=purchases side:right (==author_id)\ngroup {a.id, p.purchase_id} (\n aggregate {avg_sell = min first_published}\n)", + "WITH table_0 AS\n (\n SELECT\n MIN(published) AS _expr_0,\n author_id\n FROM albums\n GROUP BY author_id\n )\nSELECT\n a.id,\n p.purchase_id,\n MIN(table_0._expr_0) AS avg_sell\nFROM table_0\nLEFT JOIN author AS a ON table_0.author_id = a.author_id\nRIGHT JOIN purchases AS p ON table_0.author_id = p.author_id\nGROUP BY\n a.id,\n p.purchase_id", }, { - "from matches\nfilter start_date > @2023-05-30 # Some comment here\nderive [\n some_derived_value_1 = a + (b ?? 0), # And there\n some_derived_value_2 = c + some_derived_value\n]\nfilter some_derived_value_2 > 0\ngroup [country, city] (\n aggregate [\n average some_derived_value_2,\n aggr = max some_derived_value_2,\n ]\n)\nderive place = f\"{city} in {country}\"\nderive country_code = s\"LEFT(country, 2)\"\nsort [aggr, -country]\ntake 1..20", - "WITH\n table_3 AS\n (\n SELECT\n country,\n city,\n c + some_derived_value AS _expr_1\n FROM matches\n WHERE start_date > toDate('2023-05-30')\n ),\n table_1 AS\n (\n SELECT\n country,\n city,\n AVG(_expr_1) AS _expr_0,\n MAX(_expr_1) AS aggr\n FROM table_3 AS table_2\n WHERE _expr_1 > 0\n GROUP BY\n country,\n city\n )\nSELECT\n country,\n city,\n _expr_0,\n aggr,\n CONCAT(city, ' in ', country) AS place,\n LEFT(country, 2) AS country_code\nFROM table_1 AS table_0\nORDER BY\n aggr ASC,\n country DESC\nLIMIT 20", + "from matches\nfilter start_date > @2023-05-30 # Some comment here\nderive {\n some_derived_value_1 = a + (b ?? 0), # And there\n some_derived_value_2 = c + some_derived_value\n}\nfilter some_derived_value_2 > 0\ngroup {country, city} (\n aggregate {\n average some_derived_value_2,\n aggr = max some_derived_value_2\n }\n)\nderive place = f\"{city} in {country}\"\nderive country_code = s\"LEFT(country, 2)\"\nsort {aggr, -country}\ntake 1..20", + "WITH\n table_1 AS\n (\n SELECT\n country,\n city,\n c + some_derived_value AS _expr_1\n FROM matches\n WHERE start_date > toDate('2023-05-30')\n ),\n table_0 AS\n (\n SELECT\n country,\n city,\n AVG(_expr_1) AS _expr_0,\n MAX(_expr_1) AS aggr\n FROM table_1\n WHERE _expr_1 > 0\n GROUP BY\n country,\n city\n )\nSELECT\n country,\n city,\n _expr_0,\n aggr,\n CONCAT(city, ' in ', country) AS place,\n LEFT(country, 2) AS country_code\nFROM table_0\nORDER BY\n aggr ASC,\n country DESC\nLIMIT 20", }, }))); From 32cad222bf4875075d7a7e41c6095b8ae11109a3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 12 Sep 2023 17:10:57 +0300 Subject: [PATCH 25/93] KeyCondition IN function support different types --- src/Storages/MergeTree/KeyCondition.cpp | 48 ++++++++-- src/Storages/MergeTree/RPNBuilder.cpp | 58 ------------ src/Storages/MergeTree/RPNBuilder.h | 5 -- ...ndex_in_function_different_types.reference | 88 +++++++++++++++++++ ..._key_index_in_function_different_types.sql | 24 +++++ 5 files changed, 155 insertions(+), 68 deletions(-) create mode 100644 tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference create mode 100644 tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 42731bac19b..bea0111fdf3 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1253,10 +1253,21 @@ bool KeyCondition::tryPrepareSetIndex( const auto right_arg = func.getArgumentAt(1); - auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); + auto future_set = right_arg.tryGetPreparedSet(); if (!future_set) return false; + const auto & set_types = future_set->getTypes(); + size_t set_types_size = set_types.size(); + size_t indexes_mapping_size = indexes_mapping.size(); + + if (set_types_size != indexes_mapping_size) + return false; + + for (auto & index_mapping : indexes_mapping) + if (index_mapping.tuple_index >= set_types_size) + return false; + auto prepared_set = future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); if (!prepared_set) return false; @@ -1265,11 +1276,38 @@ bool KeyCondition::tryPrepareSetIndex( if (!prepared_set->hasExplicitSetElements()) return false; - prepared_set->checkColumnsNumber(left_args_count); - for (size_t i = 0; i < indexes_mapping.size(); ++i) - prepared_set->checkTypesEqual(indexes_mapping[i].tuple_index, data_types[i]); + /** Try to convert set columns to primary key columns. + * Example: SELECT id FROM test_table WHERE id IN (SELECT 1); + * In this example table `id` column has type UInt64, Set column has type UInt8. To use index + * we need to convert set column to primary key column. + */ + const auto & set_elements = prepared_set->getSetElements(); + size_t set_elements_size = set_elements.size(); + assert(set_types_size == set_elements_size); - out.set_index = std::make_shared(prepared_set->getSetElements(), std::move(indexes_mapping)); + Columns set_columns; + set_columns.reserve(set_elements_size); + + for (size_t i = 0; i < indexes_mapping_size; ++i) + { + size_t set_element_index = indexes_mapping[i].tuple_index; + const auto & set_element = set_elements[set_element_index]; + const auto & set_element_type = set_types[set_element_index]; + + ColumnPtr set_column; + try + { + set_column = castColumnAccurate({set_element, set_element_type, {}}, data_types[i]); + } + catch (...) + { + return false; + } + + set_columns.push_back(set_column); + } + + out.set_index = std::make_shared(set_columns, std::move(indexes_mapping)); return true; } diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index a0c96c13d59..29bcd8b87f1 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -351,64 +351,6 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) return nullptr; } -FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( - const std::vector & indexes_mapping, - const DataTypes & data_types) const -{ - const auto & prepared_sets = getTreeContext().getPreparedSets(); - - /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information - /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets - /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check - /// that the types it was prepared with are compatible with the types of the primary key. - auto types_match = [&indexes_mapping, &data_types](const DataTypes & set_types) - { - assert(indexes_mapping.size() == data_types.size()); - - for (size_t i = 0; i < indexes_mapping.size(); ++i) - { - if (indexes_mapping[i].tuple_index >= set_types.size()) - return false; - - auto lhs = removeNullable(recursiveRemoveLowCardinality(data_types[i])); - auto rhs = removeNullable(recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index])); - - if (!lhs->equals(*rhs)) - return false; - } - - return true; - }; - - if (prepared_sets && ast_node) - { - if (ast_node->as() || ast_node->as()) - return prepared_sets->findSubquery(ast_node->getTreeHash()); - - auto tree_hash = ast_node->getTreeHash(); - const auto & sets = prepared_sets->getSetsFromTuple(); - auto it = sets.find(tree_hash); - if (it == sets.end()) - return nullptr; - - for (const auto & future_set : it->second) - if (types_match(future_set->getTypes())) - return future_set; - } - else - { - const auto * node_without_alias = getNodeWithoutAlias(dag_node); - if (node_without_alias->column) - { - auto future_set = tryGetSetFromDAGNode(node_without_alias); - if (types_match(future_set->getTypes())) - return future_set; - } - } - - return nullptr; -} - RPNBuilderFunctionTreeNode RPNBuilderTreeNode::toFunctionNode() const { if (!isFunction()) diff --git a/src/Storages/MergeTree/RPNBuilder.h b/src/Storages/MergeTree/RPNBuilder.h index 9eeb6deefd5..f14f241cac8 100644 --- a/src/Storages/MergeTree/RPNBuilder.h +++ b/src/Storages/MergeTree/RPNBuilder.h @@ -116,11 +116,6 @@ public: /// Try get prepared set from node that match data types FutureSetPtr tryGetPreparedSet(const DataTypes & data_types) const; - /// Try get prepared set from node that match indexes mapping and data types - FutureSetPtr tryGetPreparedSet( - const std::vector & indexes_mapping, - const DataTypes & data_types) const; - /** Convert node to function node. * Node must be function before calling these method, otherwise exception is thrown. */ diff --git a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference new file mode 100644 index 00000000000..f34aad737d4 --- /dev/null +++ b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference @@ -0,0 +1,88 @@ +CreatingSets (Create sets before main query execution) + Expression ((Projection + Before ORDER BY)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 1-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Projection + Before ORDER BY)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 1-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Projection + Before ORDER BY)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 5-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Projection + Before ORDER BY)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 5-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Project names + Projection)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 1-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Project names + Projection)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 1-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Project names + Projection)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 5-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Project names + Projection)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 5-element set)) + Parts: 1/1 + Granules: 1/1 diff --git a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql new file mode 100644 index 00000000000..648a8041b71 --- /dev/null +++ b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql @@ -0,0 +1,24 @@ +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value UInt64 +) ENGINE=MergeTree ORDER BY (id, value); + +INSERT INTO test_table SELECT number, number FROM numbers(10); + +SET allow_experimental_analyzer = 0; + +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT 5); +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT '5'); +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toUInt8(number) FROM numbers(5)); +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toString(number) FROM numbers(5)); + +SET allow_experimental_analyzer = 1; + +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT 5); +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT '5'); +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toUInt8(number) FROM numbers(5)); +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toString(number) FROM numbers(5)); + +DROP TABLE test_table; From 60a63212c876995c7d7b458e767f87f7a6847c76 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 12 Sep 2023 21:26:53 +0300 Subject: [PATCH 26/93] Fixed tests --- src/Interpreters/PreparedSets.cpp | 11 +++++++---- src/Interpreters/PreparedSets.h | 8 ++++---- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index e0551dff2ad..955d8892284 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -48,7 +48,7 @@ static bool equals(const DataTypes & lhs, const DataTypes & rhs) FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} SetPtr FutureSetFromStorage::get() const { return set; } -const DataTypes & FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); } +DataTypes FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); } SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) { @@ -73,7 +73,7 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) set->finishInsert(); } -const DataTypes & FutureSetFromTuple::getTypes() const { return set->getElementsTypes(); } +DataTypes FutureSetFromTuple::getTypes() const { return set->getElementsTypes(); } SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) { @@ -138,7 +138,7 @@ void FutureSetFromSubquery::setQueryPlan(std::unique_ptr source_) set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); } -const DataTypes & FutureSetFromSubquery::getTypes() const +DataTypes FutureSetFromSubquery::getTypes() const { return set_and_key->set->getElementsTypes(); } @@ -183,7 +183,10 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) { auto set = external_table_set->buildOrderedSetInplace(context); if (set) - return set_and_key->set = set; + { + set_and_key->set = set; + return set_and_key->set; + } } auto plan = build(context); diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index b953b8470e1..e237789c63c 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -47,7 +47,7 @@ public: /// Returns set if set is ready (created and filled) or nullptr if not. virtual SetPtr get() const = 0; /// Returns set->getElementsTypes(), even if set is not created yet. - virtual const DataTypes & getTypes() const = 0; + virtual DataTypes getTypes() const = 0; /// If possible, return set with stored elements useful for PK analysis. virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; }; @@ -62,7 +62,7 @@ public: FutureSetFromStorage(SetPtr set_); SetPtr get() const override; - const DataTypes & getTypes() const override; + DataTypes getTypes() const override; SetPtr buildOrderedSetInplace(const ContextPtr &) override; private: @@ -79,7 +79,7 @@ public: SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - const DataTypes & getTypes() const override; + DataTypes getTypes() const override; private: SetPtr set; @@ -105,7 +105,7 @@ public: const Settings & settings); SetPtr get() const override; - const DataTypes & getTypes() const override; + DataTypes getTypes() const override; SetPtr buildOrderedSetInplace(const ContextPtr & context) override; std::unique_ptr build(const ContextPtr & context); From 7a7697059c6b5fb8fb0925429f2d3cf0cc31f8c9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 13 Sep 2023 17:44:38 +0300 Subject: [PATCH 27/93] Fixed tests --- src/Storages/MergeTree/KeyCondition.cpp | 20 +++++--------------- 1 file changed, 5 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index bea0111fdf3..53b7e170ea5 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1261,9 +1261,6 @@ bool KeyCondition::tryPrepareSetIndex( size_t set_types_size = set_types.size(); size_t indexes_mapping_size = indexes_mapping.size(); - if (set_types_size != indexes_mapping_size) - return false; - for (auto & index_mapping : indexes_mapping) if (index_mapping.tuple_index >= set_types_size) return false; @@ -1281,33 +1278,26 @@ bool KeyCondition::tryPrepareSetIndex( * In this example table `id` column has type UInt64, Set column has type UInt8. To use index * we need to convert set column to primary key column. */ - const auto & set_elements = prepared_set->getSetElements(); - size_t set_elements_size = set_elements.size(); - assert(set_types_size == set_elements_size); - - Columns set_columns; - set_columns.reserve(set_elements_size); + auto set_elements = prepared_set->getSetElements(); + assert(set_types_size == set_elements.size()); for (size_t i = 0; i < indexes_mapping_size; ++i) { size_t set_element_index = indexes_mapping[i].tuple_index; - const auto & set_element = set_elements[set_element_index]; const auto & set_element_type = set_types[set_element_index]; + auto & set_element = set_elements[set_element_index]; - ColumnPtr set_column; try { - set_column = castColumnAccurate({set_element, set_element_type, {}}, data_types[i]); + set_element = castColumnAccurate({set_element, set_element_type, {}}, data_types[i]); } catch (...) { return false; } - - set_columns.push_back(set_column); } - out.set_index = std::make_shared(set_columns, std::move(indexes_mapping)); + out.set_index = std::make_shared(set_elements, std::move(indexes_mapping)); return true; } From 29e4352c17d49e317914fae7e7bd01cceed62d1b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Sep 2023 19:54:39 +0300 Subject: [PATCH 28/93] Updated implementation --- src/Storages/MergeTree/KeyCondition.cpp | 46 +++++++++++++++++++------ 1 file changed, 36 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 53b7e170ea5..47521b9887b 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1278,26 +1278,52 @@ bool KeyCondition::tryPrepareSetIndex( * In this example table `id` column has type UInt64, Set column has type UInt8. To use index * we need to convert set column to primary key column. */ - auto set_elements = prepared_set->getSetElements(); - assert(set_types_size == set_elements.size()); + auto set_columns = prepared_set->getSetElements(); + assert(set_types_size == set_columns.size()); - for (size_t i = 0; i < indexes_mapping_size; ++i) + for (size_t indexes_mapping_index = 0; indexes_mapping_index < indexes_mapping_size; ++indexes_mapping_index) { - size_t set_element_index = indexes_mapping[i].tuple_index; + size_t set_element_index = indexes_mapping[indexes_mapping_index].tuple_index; const auto & set_element_type = set_types[set_element_index]; - auto & set_element = set_elements[set_element_index]; + auto & set_column = set_columns[set_element_index]; - try + bool is_set_column_nullable = set_element_type->isNullable(); + bool is_set_column_low_cardinality_nullable = set_element_type->isLowCardinalityNullable(); + + const NullMap * set_column_null_map = nullptr; + + if (is_set_column_nullable || is_set_column_low_cardinality_nullable) { - set_element = castColumnAccurate({set_element, set_element_type, {}}, data_types[i]); + if (is_set_column_low_cardinality_nullable) + set_column = set_column->convertToFullColumnIfLowCardinality(); + + set_column_null_map = &assert_cast(*set_column).getNullMapData(); } - catch (...) + + auto nullable_set_column = castColumnAccurateOrNull({set_column, set_element_type, {}}, data_types[indexes_mapping_index]); + const auto & nullable_set_column_typed = assert_cast(*nullable_set_column); + const auto & nullable_set_column_null_map = nullable_set_column_typed.getNullMapData(); + size_t nullable_set_column_null_map_size = nullable_set_column_null_map.size(); + + IColumn::Filter filter(nullable_set_column_null_map_size); + + if (set_column_null_map) { - return false; + for (size_t i = 0; i < nullable_set_column_null_map_size; ++i) + filter[i] = (*set_column_null_map)[i] || !nullable_set_column_null_map[i]; + + set_column = nullable_set_column_typed.filter(filter, 0); + } + else + { + for (size_t i = 0; i < nullable_set_column_null_map_size; ++i) + filter[i] = !nullable_set_column_null_map[i]; + + set_column = nullable_set_column_typed.getNestedColumn().filter(filter, 0); } } - out.set_index = std::make_shared(set_elements, std::move(indexes_mapping)); + out.set_index = std::make_shared(set_columns, std::move(indexes_mapping)); return true; } From 7ce3a587ca2aff76379d1e4f360721694cb966c0 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 15 Sep 2023 20:52:21 +0200 Subject: [PATCH 29/93] fix build --- src/IO/S3/Client.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index ceae33aee25..21d87c58d20 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -558,10 +558,10 @@ Client::doRequestWithRetryNetworkErrors(const RequestType & request, RequestFn r auto with_retries = [this, request_fn_ = std::move(request_fn)] (const RequestType & request_) { chassert(client_configuration.retryStrategy); - const long max_attempts = client_configuration.retryStrategy->GetMaxAttempts(); + const Int64 max_attempts = client_configuration.retryStrategy->GetMaxAttempts(); chassert(max_attempts > 0); std::exception_ptr last_exception = nullptr; - for (long attempt_no = 0; attempt_no < max_attempts; ++attempt_no) + for (Int64 attempt_no = 0; attempt_no < max_attempts; ++attempt_no) { try { From 8ef910e1fa0c4d8d6053b111efab5b4aecaaffb2 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 15 Sep 2023 22:34:09 +0200 Subject: [PATCH 30/93] Updated to check if cleanup is enabled before adding to cleanedup_rows_count --- .../Algorithms/ReplacingSortedAlgorithm.cpp | 4 ++-- ...lacing_merge_tree_vertical_merge.reference | 3 +++ ...77_replacing_merge_tree_vertical_merge.sql | 21 ++++++++++++++++++- 3 files changed, 25 insertions(+), 3 deletions(-) diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp index 4d2443b1e46..18f144bee3d 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp @@ -83,7 +83,7 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() uint8_t value = assert_cast(*(*selected_row.all_columns)[is_deleted_column_number]).getData()[selected_row.row_num]; if (!cleanup || !value) insertRow(); - else if (cleanedup_rows_count != nullptr) + else if (cleanup && cleanedup_rows_count != nullptr) *cleanedup_rows_count += current_row_sources.size(); } else @@ -141,7 +141,7 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() uint8_t value = assert_cast(*(*selected_row.all_columns)[is_deleted_column_number]).getData()[selected_row.row_num]; if (!cleanup || !value) insertRow(); - else if (cleanedup_rows_count != nullptr) + else if (cleanup && cleanedup_rows_count != nullptr) *cleanedup_rows_count += current_row_sources.size(); } else diff --git a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference index e0f8c3bae3f..6bac6173183 100644 --- a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference +++ b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference @@ -2,3 +2,6 @@ 2018-01-01 1 1 2018-01-01 2 2 2018-01-01 2 2 +== (Replicas) Test optimize == +d2 1 0 +d4 1 0 diff --git a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql index 8c51a6f34da..931297fdd3b 100644 --- a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql +++ b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql @@ -1,10 +1,29 @@ set optimize_on_insert = 0; drop table if exists tab_00577; -create table tab_00577 (date Date, version UInt64, val UInt64) engine = ReplacingMergeTree(version) partition by date order by date settings enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 0; +create table tab_00577 (date Date, version UInt64, val UInt64) engine = ReplacingMergeTree(version) partition by date order by date settings enable_vertical_merge_algorithm = 1, + vertical_merge_algorithm_min_rows_to_activate = 0, vertical_merge_algorithm_min_columns_to_activate = 0, min_rows_for_wide_part = 0, + min_bytes_for_wide_part = 0; insert into tab_00577 values ('2018-01-01', 2, 2), ('2018-01-01', 1, 1); insert into tab_00577 values ('2018-01-01', 0, 0); select * from tab_00577 order by version; OPTIMIZE TABLE tab_00577; select * from tab_00577; drop table tab_00577; + + +DROP TABLE IF EXISTS testCleanupR1; +CREATE TABLE testCleanupR1 (uid String, version UInt32, is_deleted UInt8) + ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{database}/tables/test_cleanup/', 'r1', version, is_deleted) + ORDER BY uid SETTINGS enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 0; +INSERT INTO testCleanupR1 (*) VALUES ('d1', 1, 0),('d2', 1, 0),('d3', 1, 0),('d4', 1, 0); +INSERT INTO testCleanupR1 (*) VALUES ('d3', 2, 1); +INSERT INTO testCleanupR1 (*) VALUES ('d1', 2, 1); +SYSTEM SYNC REPLICA testCleanupR1; -- Avoid "Cannot select parts for optimization: Entry for part all_2_2_0 hasn't been read from the replication log yet" + +OPTIMIZE TABLE testCleanupR1 FINAL CLEANUP; + +-- Only d3 to d5 remain +SELECT '== (Replicas) Test optimize =='; +SELECT * FROM testCleanupR1 order by uid; +DROP TABLE IF EXISTS testCleanupR1 \ No newline at end of file From 745006a43331ba9a48595b91a59a5725cd732917 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Fri, 15 Sep 2023 23:37:40 +0000 Subject: [PATCH 31/93] add missing documentation for system sync database replica. Signed-off-by: Jianfei Hu --- docs/en/sql-reference/statements/system.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 443db7c5ac2..f2494bfddde 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -340,6 +340,11 @@ After running this statement the `[db.]replicated_merge_tree_family_table_name` - If a `LIGHTWEIGHT` modifier was specified then the query waits only for `GET_PART`, `ATTACH_PART`, `DROP_RANGE`, `REPLACE_RANGE` and `DROP_PART` entries to be processed. - If a `PULL` modifier was specified then the query pulls new replication queue entries from ZooKeeper, but does not wait for anything to be processed. +### SYNC DATABASE REPLICA + +Sync all tables from all [replicated database](https://clickhouse.com/docs/en/engines/database-engines/replicated). This will make the ClickHouse synchronously fetch all table definition +from zookeeper and applied locally. + ### RESTART REPLICA Provides possibility to reinitialize Zookeeper session's state for `ReplicatedMergeTree` table, will compare current state with Zookeeper as source of truth and add tasks to Zookeeper queue if needed. From 9f009cccd5d01be29ff8e8ab6063297ec2a73b46 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 17 Sep 2023 15:22:51 +0000 Subject: [PATCH 32/93] Incorporate review feedback --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 18 ++++++++++-------- .../MergeTree/MergeTreeIndexUSearch.cpp | 18 ++++++++++-------- 2 files changed, 20 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index f00f11359e1..d15d89ad6f9 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -160,21 +160,23 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t const auto & column_array_offsets = column_array->getOffsets(); const size_t num_rows = column_array_offsets.size(); - /// The index dimension is inferred from the inserted arrays (array cardinality). If no value was specified in the INSERT statement - /// for the annoy-indexed column (i.e. default value), we have a problem. Reject such values. - if (column_array_offsets.empty() || column_array_offsets[0] == 0) - /// (The if condition is a bit weird but I have seen either with default values) - throw Exception(ErrorCodes::INCORRECT_DATA, "Tried to insert {} rows into Annoy index but there were no values to insert. Likely, the INSERT used default values - these are not supported for Annoy.", rows_read); + /// The Annoy algorithm naturally assumes that the indexed vectors have dimension >= 0. This condition is violated if empty arrays + /// are INSERTed into an Annoy-indexed column or if no value was specified at all in which case the arrays take on their default + /// value which is also an empty array. + if (column_array->isDefaultAt(0)) + throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); /// Check all sizes are the same size_t dimension = column_array_offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension) - throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name); - /// Also check that previously inserted blocks have the same size as this block + /// Also check that previously inserted blocks have the same size as this block. + /// Note that this guarantees consistency of dimension only within parts. We are unable to detect inconsistent dimensions across + /// parts - for this, a little help from the user is needed, e.g. CONSTRAINT cnstr CHECK length(array) = 42. if (index && index->getDimensions() != dimension) - throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name); if (!index) index = std::make_shared>(dimension); diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index 9531b9188bf..a00cab6ca59 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -179,21 +179,23 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t const auto & column_array_offsets = column_array->getOffsets(); const size_t num_rows = column_array_offsets.size(); - /// The index dimension is inferred from the inserted arrays (array cardinality). If no value was specified in the INSERT statement - /// for the usearch-indexed column (i.e. default value), we have a problem. Reject such values. - if (column_array_offsets.empty() || column_array_offsets[0] == 0) - /// (The if condition is a bit weird but I have seen either with default values) - throw Exception(ErrorCodes::INCORRECT_DATA, "Tried to insert {} rows into usearch index but there were no values to insert. Likely, the INSERT used default values - these are not supported for Annoy.", rows_read); + /// The Usearch algorithm naturally assumes that the indexed vectors have dimension >= 0. This condition is violated if empty arrays + /// are INSERTed into an Usearch-indexed column or if no value was specified at all in which case the arrays take on their default + /// value which is also an empty array. + if (column_array->isDefaultAt(0)) + throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); /// Check all sizes are the same size_t dimension = column_array_offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension) - throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name); - /// Also check that previously inserted blocks have the same size as this block + /// Also check that previously inserted blocks have the same size as this block. + /// Note that this guarantees consistency of dimension only within parts. We are unable to detect inconsistent dimensions across + /// parts - for this, a little help from the user is needed, e.g. CONSTRAINT cnstr CHECK length(array) = 42. if (index && index->getDimensions() != dimension) - throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name); if (!index) index = std::make_shared>(dimension, scalar_kind); From de4f22e20aa1c4d044a525474f5ed2d03e6c59a5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 17 Sep 2023 15:26:36 +0000 Subject: [PATCH 33/93] Typo --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeIndexUSearch.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index d15d89ad6f9..3eec8614dcd 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -160,9 +160,9 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t const auto & column_array_offsets = column_array->getOffsets(); const size_t num_rows = column_array_offsets.size(); - /// The Annoy algorithm naturally assumes that the indexed vectors have dimension >= 0. This condition is violated if empty arrays + /// The Annoy algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays /// are INSERTed into an Annoy-indexed column or if no value was specified at all in which case the arrays take on their default - /// value which is also an empty array. + /// value which is also empty. if (column_array->isDefaultAt(0)) throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index a00cab6ca59..009c004faea 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -179,9 +179,9 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t const auto & column_array_offsets = column_array->getOffsets(); const size_t num_rows = column_array_offsets.size(); - /// The Usearch algorithm naturally assumes that the indexed vectors have dimension >= 0. This condition is violated if empty arrays + /// The Usearch algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays /// are INSERTed into an Usearch-indexed column or if no value was specified at all in which case the arrays take on their default - /// value which is also an empty array. + /// values which is also empty. if (column_array->isDefaultAt(0)) throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); From 46fa7dbb8080aa9a2189a1fe0adbe3110f89c11f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 5 Sep 2023 12:59:32 +0000 Subject: [PATCH 34/93] add libFuzzer build on 'libFuzzer' label, build produces artifacts --- .github/workflows/pull_request.yml | 45 +++++++++++ CMakeLists.txt | 22 ++---- cmake/utils.cmake | 120 +++++++++++++++++++++++++++++ docker/packager/binary/build.sh | 8 +- 4 files changed, 173 insertions(+), 22 deletions(-) create mode 100644 cmake/utils.cmake diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index ce135846dd5..e6a4d1bf92e 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5059,6 +5059,51 @@ jobs: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" ############################################################################################# +#################################### libFuzzer build ######################################## +############################################################################################# + BuilderFuzzers: + if: contains(github.event.pull_request.labels.*.name, 'libFuzzer') + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=fuzzers + EOF + - name: Download changed images + uses: actions/download-artifact@v3 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + submodules: true + - name: Build + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v3 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" +############################################################################################# ###################################### JEPSEN TESTS ######################################### ############################################################################################# Jepsen: diff --git a/CMakeLists.txt b/CMakeLists.txt index 65ff9dc5384..781a9efe64a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -19,6 +19,7 @@ include (cmake/tools.cmake) include (cmake/ccache.cmake) include (cmake/clang_tidy.cmake) include (cmake/git.cmake) +include (cmake/utils.cmake) # Ignore export() since we don't use it, # but it gets broken with a global targets via link_libraries() @@ -562,22 +563,6 @@ add_subdirectory (programs) add_subdirectory (tests) add_subdirectory (utils) -# Function get_all_targets collects all targets recursively -function(get_all_targets var) - macro(get_all_targets_recursive targets dir) - get_property(subdirectories DIRECTORY ${dir} PROPERTY SUBDIRECTORIES) - foreach(subdir ${subdirectories}) - get_all_targets_recursive(${targets} ${subdir}) - endforeach() - get_property(current_targets DIRECTORY ${dir} PROPERTY BUILDSYSTEM_TARGETS) - list(APPEND ${targets} ${current_targets}) - endmacro() - - set(targets) - get_all_targets_recursive(targets ${CMAKE_CURRENT_SOURCE_DIR}) - set(${var} ${targets} PARENT_SCOPE) -endfunction() - if (FUZZER) # Bundle fuzzers target add_custom_target(fuzzers) @@ -592,11 +577,14 @@ if (FUZZER) # clickhouse fuzzer isn't working correctly # initial PR https://github.com/ClickHouse/ClickHouse/pull/27526 #if (target MATCHES ".+_fuzzer" OR target STREQUAL "clickhouse") - if (target MATCHES ".+_fuzzer") + if (target_type STREQUAL "EXECUTABLE" AND target MATCHES ".+_fuzzer") message(STATUS "${target} instrumented with fuzzer") target_link_libraries(${target} PUBLIC ch_contrib::fuzzer) # Add to fuzzers bundle add_dependencies(fuzzers ${target}) + get_target_filename(${target} target_bin_name) + get_target_property(target_bin_dir ${target} BINARY_DIR) + add_custom_command(TARGET fuzzers POST_BUILD COMMAND mv "${target_bin_dir}/${target_bin_name}" "${CMAKE_CURRENT_BINARY_DIR}/programs/" VERBATIM) endif() endif() endforeach() diff --git a/cmake/utils.cmake b/cmake/utils.cmake new file mode 100644 index 00000000000..a318408098a --- /dev/null +++ b/cmake/utils.cmake @@ -0,0 +1,120 @@ +# Useful stuff + +# Function get_all_targets collects all targets recursively +function(get_all_targets outvar) + macro(get_all_targets_recursive targets dir) + get_property(subdirectories DIRECTORY ${dir} PROPERTY SUBDIRECTORIES) + foreach(subdir ${subdirectories}) + get_all_targets_recursive(${targets} ${subdir}) + endforeach() + get_property(current_targets DIRECTORY ${dir} PROPERTY BUILDSYSTEM_TARGETS) + list(APPEND ${targets} ${current_targets}) + endmacro() + + set(targets) + get_all_targets_recursive(targets ${CMAKE_CURRENT_SOURCE_DIR}) + set(${outvar} ${targets} PARENT_SCOPE) +endfunction() + + +# Function get_target_filename calculates target's output file name +function(get_target_filename target outvar) + get_target_property(prop_type "${target}" TYPE) + get_target_property(prop_is_framework "${target}" FRAMEWORK) + get_target_property(prop_outname "${target}" OUTPUT_NAME) + get_target_property(prop_archive_outname "${target}" ARCHIVE_OUTPUT_NAME) + get_target_property(prop_library_outname "${target}" LIBRARY_OUTPUT_NAME) + get_target_property(prop_runtime_outname "${target}" RUNTIME_OUTPUT_NAME) + # message("prop_archive_outname: ${prop_archive_outname}") + # message("prop_library_outname: ${prop_library_outname}") + # message("prop_runtime_outname: ${prop_runtime_outname}") + if(DEFINED CMAKE_BUILD_TYPE) + get_target_property(prop_cfg_outname "${target}" "${OUTPUT_NAME}_${CMAKE_BUILD_TYPE}") + get_target_property(prop_archive_cfg_outname "${target}" "${ARCHIVE_OUTPUT_NAME}_${CMAKE_BUILD_TYPE}") + get_target_property(prop_library_cfg_outname "${target}" "${LIBRARY_OUTPUT_NAME}_${CMAKE_BUILD_TYPE}") + get_target_property(prop_runtime_cfg_outname "${target}" "${RUNTIME_OUTPUT_NAME}_${CMAKE_BUILD_TYPE}") + # message("prop_archive_cfg_outname: ${prop_archive_cfg_outname}") + # message("prop_library_cfg_outname: ${prop_library_cfg_outname}") + # message("prop_runtime_cfg_outname: ${prop_runtime_cfg_outname}") + if(NOT ("${prop_cfg_outname}" STREQUAL "prop_cfg_outname-NOTFOUND")) + set(prop_outname "${prop_cfg_outname}") + endif() + if(NOT ("${prop_archive_cfg_outname}" STREQUAL "prop_archive_cfg_outname-NOTFOUND")) + set(prop_archive_outname "${prop_archive_cfg_outname}") + endif() + if(NOT ("${prop_library_cfg_outname}" STREQUAL "prop_library_cfg_outname-NOTFOUND")) + set(prop_library_outname "${prop_library_cfg_outname}") + endif() + if(NOT ("${prop_runtime_cfg_outname}" STREQUAL "prop_runtime_cfg_outname-NOTFOUND")) + set(prop_runtime_outname "${prop_runtime_cfg_outname}") + endif() + endif() + set(outname "${target}") + if(NOT ("${prop_outname}" STREQUAL "prop_outname-NOTFOUND")) + set(outname "${prop_outname}") + endif() + if("${prop_is_framework}") + set(filename "${outname}") + elseif(prop_type STREQUAL "STATIC_LIBRARY") + if(NOT ("${prop_archive_outname}" STREQUAL "prop_archive_outname-NOTFOUND")) + set(outname "${prop_archive_outname}") + endif() + set(filename "${CMAKE_STATIC_LIBRARY_PREFIX}${outname}${CMAKE_STATIC_LIBRARY_SUFFIX}") + elseif(prop_type STREQUAL "MODULE_LIBRARY") + if(NOT ("${prop_library_outname}" STREQUAL "prop_library_outname-NOTFOUND")) + set(outname "${prop_library_outname}") + endif() + set(filename "${CMAKE_SHARED_MODULE_LIBRARY_PREFIX}${outname}${CMAKE_SHARED_MODULE_LIBRARY_SUFFIX}") + elseif(prop_type STREQUAL "SHARED_LIBRARY") + if(WIN32) + if(NOT ("${prop_runtime_outname}" STREQUAL "prop_runtime_outname-NOTFOUND")) + set(outname "${prop_runtime_outname}") + endif() + else() + if(NOT ("${prop_library_outname}" STREQUAL "prop_library_outname-NOTFOUND")) + set(outname "${prop_library_outname}") + endif() + endif() + set(filename "${CMAKE_SHARED_LIBRARY_PREFIX}${outname}${CMAKE_SHARED_LIBRARY_SUFFIX}") + elseif(prop_type STREQUAL "EXECUTABLE") + if(NOT ("${prop_runtime_outname}" STREQUAL "prop_runtime_outname-NOTFOUND")) + set(outname "${prop_runtime_outname}") + endif() + set(filename "${CMAKE_EXECUTABLE_PREFIX}${outname}${CMAKE_EXECUTABLE_SUFFIX}") + else() + message(FATAL_ERROR "target \"${target}\" is not of type STATIC_LIBRARY, MODULE_LIBRARY, SHARED_LIBRARY, or EXECUTABLE.") + endif() + set("${outvar}" "${filename}" PARENT_SCOPE) +endfunction() + + +# Function get_cmake_properties returns list of all propreties that cmake supports +function(get_cmake_properties outvar) + execute_process(COMMAND cmake --help-property-list OUTPUT_VARIABLE cmake_properties) + # Convert command output into a CMake list + string(REGEX REPLACE ";" "\\\\;" cmake_properties "${cmake_properties}") + string(REGEX REPLACE "\n" ";" cmake_properties "${cmake_properties}") + list(REMOVE_DUPLICATES cmake_properties) + set("${outvar}" "${cmake_properties}" PARENT_SCOPE) +endfunction() + +# Function get_target_property_list returns list of all propreties set for target +function(get_target_property_list target outvar) + get_cmake_properties(cmake_property_list) + foreach(property ${cmake_property_list}) + string(REPLACE "" "${CMAKE_BUILD_TYPE}" property ${property}) + + # https://stackoverflow.com/questions/32197663/how-can-i-remove-the-the-location-property-may-not-be-read-from-target-error-i + if(property STREQUAL "LOCATION" OR property MATCHES "^LOCATION_" OR property MATCHES "_LOCATION$") + continue() + endif() + + get_property(was_set TARGET ${target} PROPERTY ${property} SET) + if(was_set) + get_target_property(value ${target} ${property}) + string(REGEX REPLACE ";" "\\\\\\\\;" value "${value}") + list(APPEND outvar "${property} = ${value}") + endif() + endforeach() + set(${outvar} ${${outvar}} PARENT_SCOPE) +endfunction() diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 39d299e1794..75a18528e65 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -97,11 +97,9 @@ if [ -n "$MAKE_DEB" ]; then bash -x /build/packages/build fi -if [ "$BUILD_TARGET" != "fuzzers" ]; then - mv ./programs/clickhouse* /output - [ -x ./programs/self-extracting/clickhouse ] && mv ./programs/self-extracting/clickhouse /output - mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds -fi +mv ./programs/clickhouse* /output || mv ./programs/*_fuzzer /output +[ -x ./programs/self-extracting/clickhouse ] && mv ./programs/self-extracting/clickhouse /output +mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds prepare_combined_output () { local OUTPUT From b7a17bf8dda0b94db456d2883e507d503f400594 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 10 Sep 2023 17:07:49 +0000 Subject: [PATCH 35/93] add libFuzzer tests, initial integration --- .github/workflows/pull_request.yml | 37 ++ docker/test/libfuzzer/Dockerfile | 42 +++ docker/test/libfuzzer/parse_options.py | 61 ++++ docker/test/libfuzzer/run_libfuzzer.sh | 115 +++++++ tests/ci/build_download_helper.py | 6 + tests/ci/ci_config.py | 1 + tests/ci/libfuzzer_test_check.py | 458 +++++++++++++++++++++++++ 7 files changed, 720 insertions(+) create mode 100644 docker/test/libfuzzer/Dockerfile create mode 100644 docker/test/libfuzzer/parse_options.py create mode 100644 docker/test/libfuzzer/run_libfuzzer.sh create mode 100644 tests/ci/libfuzzer_test_check.py diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index e6a4d1bf92e..7e56254bac0 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5103,6 +5103,43 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" +############################################################################################## +################################ libFuzzer TESTS ############################################# +############################################################################################## + libFuzzerTest: + needs: [BuilderFuzzers] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/libfuzzer + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=libFuzzer tests + REPO_COPY=${{runner.temp}}/libfuzzer/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: libFuzzer test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 libfuzzer_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" ############################################################################################# ###################################### JEPSEN TESTS ######################################### ############################################################################################# diff --git a/docker/test/libfuzzer/Dockerfile b/docker/test/libfuzzer/Dockerfile new file mode 100644 index 00000000000..77815431314 --- /dev/null +++ b/docker/test/libfuzzer/Dockerfile @@ -0,0 +1,42 @@ +ARG FROM_TAG=latest +FROM clickhouse/test-base:$FROM_TAG + +# ARG for quick switch to a given ubuntu mirror +ARG apt_archive="http://archive.ubuntu.com" +RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list + +ENV LANG=C.UTF-8 +ENV TZ=Europe/Amsterdam +RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone + +RUN apt-get update \ + && DEBIAN_FRONTEND=noninteractive apt-get install --yes --no-install-recommends \ + ca-certificates \ + libc6-dbg \ + moreutils \ + ncdu \ + p7zip-full \ + parallel \ + psmisc \ + python3 \ + python3-pip \ + rsync \ + tree \ + tzdata \ + vim \ + wget \ + && apt-get autoremove --yes \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + +RUN pip3 install Jinja2 + +COPY * / + +SHELL ["/bin/bash", "-c"] +CMD set -o pipefail \ + && cd /workspace \ + && timeout -s 9 1h /run_libfuzzer.sh 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee main.log + +# docker run --network=host --volume :/workspace -e PR_TO_TEST=<> -e SHA_TO_TEST=<> clickhouse/libfuzzer + diff --git a/docker/test/libfuzzer/parse_options.py b/docker/test/libfuzzer/parse_options.py new file mode 100644 index 00000000000..5695e80a714 --- /dev/null +++ b/docker/test/libfuzzer/parse_options.py @@ -0,0 +1,61 @@ +#!/usr/bin/env python3 +# Copyright 2020 Google LLC +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +################################################################################ +"""Helper script for parsing custom fuzzing options.""" +import configparser +import sys + + +def parse_options(options_file_path, options_section): + """Parses the given file and returns options from the given section.""" + parser = configparser.ConfigParser() + parser.read(options_file_path) + + if not parser.has_section(options_section): + return None + + options = parser[options_section] + + if options_section == "libfuzzer": + options_string = " ".join( + "-%s=%s" % (key, value) for key, value in options.items() + ) + else: + # Sanitizer options. + options_string = ":".join( + "%s=%s" % (key, value) for key, value in options.items() + ) + + return options_string + + +def main(): + """Processes the arguments and prints the options in the correct format.""" + if len(sys.argv) < 3: + sys.stderr.write( + "Usage: %s \n" % sys.argv[0] + ) + return 1 + + options = parse_options(sys.argv[1], sys.argv[2]) + if options is not None: + print(options) + + return 0 + + +if __name__ == "__main__": + sys.exit(main()) diff --git a/docker/test/libfuzzer/run_libfuzzer.sh b/docker/test/libfuzzer/run_libfuzzer.sh new file mode 100644 index 00000000000..49a59dafb90 --- /dev/null +++ b/docker/test/libfuzzer/run_libfuzzer.sh @@ -0,0 +1,115 @@ +#!/bin/bash -eu +# Copyright 2016 Google Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +################################################################################ + +# Fuzzer runner. Appends .options arguments and seed corpus to users args. +# Usage: $0 + +export PATH=$OUT:$PATH +cd $OUT + +DEBUGGER=${DEBUGGER:-} + +FUZZER=$1 +shift + +# This env var is set by CIFuzz. CIFuzz fills this directory with the corpus +# from ClusterFuzz. +CORPUS_DIR=${CORPUS_DIR:-} +if [ -z "$CORPUS_DIR" ] +then + CORPUS_DIR="/tmp/${FUZZER}_corpus" + rm -rf $CORPUS_DIR && mkdir -p $CORPUS_DIR +fi + +SANITIZER=${SANITIZER:-} +if [ -z $SANITIZER ]; then + # If $SANITIZER is not specified (e.g. calling from `reproduce` command), it + # is not important and can be set to any value. + SANITIZER="default" +fi + +if [[ "$RUN_FUZZER_MODE" = interactive ]]; then + FUZZER_OUT="$OUT/${FUZZER}_${FUZZING_ENGINE}_${SANITIZER}_out" +else + FUZZER_OUT="/tmp/${FUZZER}_${FUZZING_ENGINE}_${SANITIZER}_out" +fi + + +rm -rf $FUZZER_OUT && mkdir -p $FUZZER_OUT + +SEED_CORPUS="${FUZZER}_seed_corpus.zip" + +# TODO: Investigate why this code block is skipped +# by all default fuzzers in bad_build_check. +# They all set SKIP_SEED_CORPUS=1. +if [ -f $SEED_CORPUS ] && [ -z ${SKIP_SEED_CORPUS:-} ]; then + echo "Using seed corpus: $SEED_CORPUS" + unzip -o -d ${CORPUS_DIR}/ $SEED_CORPUS > /dev/null +fi + +OPTIONS_FILE="${FUZZER}.options" +CUSTOM_LIBFUZZER_OPTIONS="" + +if [ -f $OPTIONS_FILE ]; then + custom_asan_options=$(parse_options.py $OPTIONS_FILE asan) + if [ ! -z $custom_asan_options ]; then + export ASAN_OPTIONS="$ASAN_OPTIONS:$custom_asan_options" + fi + + custom_msan_options=$(parse_options.py $OPTIONS_FILE msan) + if [ ! -z $custom_msan_options ]; then + export MSAN_OPTIONS="$MSAN_OPTIONS:$custom_msan_options" + fi + + custom_ubsan_options=$(parse_options.py $OPTIONS_FILE ubsan) + if [ ! -z $custom_ubsan_options ]; then + export UBSAN_OPTIONS="$UBSAN_OPTIONS:$custom_ubsan_options" + fi + + CUSTOM_LIBFUZZER_OPTIONS=$(parse_options.py $OPTIONS_FILE libfuzzer) +fi + + + +CMD_LINE="$OUT/$FUZZER $FUZZER_ARGS $*" + +if [ -z ${SKIP_SEED_CORPUS:-} ]; then +CMD_LINE="$CMD_LINE $CORPUS_DIR" +fi + +if [[ ! -z ${CUSTOM_LIBFUZZER_OPTIONS} ]]; then +CMD_LINE="$CMD_LINE $CUSTOM_LIBFUZZER_OPTIONS" +fi + +if [[ ! "$CMD_LINE" =~ "-dict=" ]]; then +if [ -f "$FUZZER.dict" ]; then + CMD_LINE="$CMD_LINE -dict=$FUZZER.dict" +fi +fi + +CMD_LINE="$CMD_LINE < /dev/null" + +echo $CMD_LINE + +# Unset OUT so the fuzz target can't rely on it. +unset OUT + +if [ ! -z "$DEBUGGER" ]; then + CMD_LINE="$DEBUGGER $CMD_LINE" +fi + +bash -c "$CMD_LINE" diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index a6fda749494..02e22e88a96 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -210,3 +210,9 @@ def download_performance_build(check_name, reports_path, result_path): result_path, lambda x: x.endswith("performance.tar.zst"), ) + + +def download_fuzzers(check_name, reports_path, result_path): + download_builds_filter( + check_name, reports_path, result_path, lambda x: x.endswith("_fuzzer") + ) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index b9ccc23cb2e..198395eca27 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -282,6 +282,7 @@ CI_CONFIG = CiConfig( "SQLancer (debug)": TestConfig("package_debug"), "Sqllogic test (release)": TestConfig("package_release"), "SQLTest": TestConfig("package_release"), + "libFuzzer tests": TestConfig("fuzzers"), }, ) CI_CONFIG.validate() diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py new file mode 100644 index 00000000000..148b6e6d1e4 --- /dev/null +++ b/tests/ci/libfuzzer_test_check.py @@ -0,0 +1,458 @@ +#!/usr/bin/env python3 + +import argparse +import csv +import logging +import os +import re +import subprocess +import sys +import atexit +from pathlib import Path +from typing import List, Tuple + +from github import Github + +# from build_download_helper import download_all_deb_packages +from build_download_helper import download_fuzzers +from clickhouse_helper import ( + CiLogsCredentials, + # ClickHouseHelper, + # prepare_tests_results_for_clickhouse, +) +from commit_status_helper import ( + # NotSet, + RerunHelper, + get_commit, + # override_status, + # post_commit_status, + # post_commit_status_to_file, + update_mergeable_check, +) +from docker_pull_helper import DockerImage # , get_image_with_version + +# from download_release_packages import download_last_release +from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH +from get_robot_token import get_best_robot_token +from pr_info import PRInfo # , FORCE_TESTS_LABEL +from report import TestResults, read_test_results + +# from s3_helper import S3Helper +from stopwatch import Stopwatch + +# from tee_popen import TeePopen +# from upload_result_helper import upload_results + +NO_CHANGES_MSG = "Nothing to run" + + +def get_additional_envs(check_name, run_by_hash_num, run_by_hash_total): + result = [] + if "DatabaseReplicated" in check_name: + result.append("USE_DATABASE_REPLICATED=1") + if "DatabaseOrdinary" in check_name: + result.append("USE_DATABASE_ORDINARY=1") + if "wide parts enabled" in check_name: + result.append("USE_POLYMORPHIC_PARTS=1") + if "ParallelReplicas" in check_name: + result.append("USE_PARALLEL_REPLICAS=1") + if "s3 storage" in check_name: + result.append("USE_S3_STORAGE_FOR_MERGE_TREE=1") + if "analyzer" in check_name: + result.append("USE_NEW_ANALYZER=1") + + if run_by_hash_total != 0: + result.append(f"RUN_BY_HASH_NUM={run_by_hash_num}") + result.append(f"RUN_BY_HASH_TOTAL={run_by_hash_total}") + + return result + + +# def get_image_name(check_name): +# if "stateless" in check_name.lower(): +# return "clickhouse/stateless-test" +# if "stateful" in check_name.lower(): +# return "clickhouse/stateful-test" +# else: +# raise Exception(f"Cannot deduce image name based on check name {check_name}") + + +def get_run_command( + # check_name: str, + fuzzers_path: str, + repo_path: str, + result_path: str, + # server_log_path: str, + kill_timeout: int, + additional_envs: List[str], + ci_logs_args: str, + image: DockerImage, + # flaky_check: bool, + # tests_to_run: List[str], +) -> str: + additional_options = ["--hung-check"] + additional_options.append("--print-time") + + # if tests_to_run: + # additional_options += tests_to_run + + additional_options_str = ( + '-e ADDITIONAL_OPTIONS="' + " ".join(additional_options) + '"' + ) + + envs = [ + f"-e MAX_RUN_TIME={int(0.9 * kill_timeout)}", + # a static link, don't use S3_URL or S3_DOWNLOAD + '-e S3_URL="https://s3.amazonaws.com/clickhouse-datasets"', + ] + + # if flaky_check: + # envs.append("-e NUM_TRIES=100") + # envs.append("-e MAX_RUN_TIME=1800") + + envs += [f"-e {e}" for e in additional_envs] + + env_str = " ".join(envs) + # volume_with_broken_test = ( + # f"--volume={repo_path}/tests/analyzer_tech_debt.txt:/analyzer_tech_debt.txt " + # if "analyzer" in check_name + # else "" + # ) + + return ( + f"docker run --volume={fuzzers_path}:/fuzzers " + f"{ci_logs_args}" + f"--volume={repo_path}/tests:/usr/share/clickhouse-test " + # f"{volume_with_broken_test}" + f"--volume={result_path}:/test_output " + # f"--volume={server_log_path}:/var/log/clickhouse-server " + f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image}" + ) + + +def get_tests_to_run(pr_info: PRInfo) -> List[str]: + result = set() + + if pr_info.changed_files is None: + return [] + + for fpath in pr_info.changed_files: + if re.match(r"tests/queries/0_stateless/[0-9]{5}", fpath): + logging.info("File '%s' is changed and seems like a test", fpath) + fname = fpath.split("/")[3] + fname_without_ext = os.path.splitext(fname)[0] + # add '.' to the end of the test name not to run all tests with the same prefix + # e.g. we changed '00001_some_name.reference' + # and we have ['00001_some_name.sh', '00001_some_name_2.sql'] + # so we want to run only '00001_some_name.sh' + result.add(fname_without_ext + ".") + elif "tests/queries/" in fpath: + # log suspicious changes from tests/ for debugging in case of any problems + logging.info("File '%s' is changed, but it doesn't look like a test", fpath) + return list(result) + + +def process_results( + result_folder: str, + server_log_path: str, +) -> Tuple[str, str, TestResults, List[str]]: + test_results = [] # type: TestResults + additional_files = [] + # Just upload all files from result_folder. + # If task provides processed results, then it's responsible for content of result_folder. + if os.path.exists(result_folder): + test_files = [ + f + for f in os.listdir(result_folder) + if os.path.isfile(os.path.join(result_folder, f)) + ] + additional_files = [os.path.join(result_folder, f) for f in test_files] + + if os.path.exists(server_log_path): + server_log_files = [ + f + for f in os.listdir(server_log_path) + if os.path.isfile(os.path.join(server_log_path, f)) + ] + additional_files = additional_files + [ + os.path.join(server_log_path, f) for f in server_log_files + ] + + status = [] + status_path = os.path.join(result_folder, "check_status.tsv") + if os.path.exists(status_path): + logging.info("Found test_results.tsv") + with open(status_path, "r", encoding="utf-8") as status_file: + status = list(csv.reader(status_file, delimiter="\t")) + + if len(status) != 1 or len(status[0]) != 2: + logging.info("Files in result folder %s", os.listdir(result_folder)) + return "error", "Invalid check_status.tsv", test_results, additional_files + state, description = status[0][0], status[0][1] + + try: + results_path = Path(result_folder) / "test_results.tsv" + + if results_path.exists(): + logging.info("Found test_results.tsv") + else: + logging.info("Files in result folder %s", os.listdir(result_folder)) + return "error", "Not found test_results.tsv", test_results, additional_files + + test_results = read_test_results(results_path) + if len(test_results) == 0: + return "error", "Empty test_results.tsv", test_results, additional_files + except Exception as e: + return ( + "error", + f"Cannot parse test_results.tsv ({e})", + test_results, + additional_files, + ) + + return state, description, test_results, additional_files + + +def parse_args(): + parser = argparse.ArgumentParser() + parser.add_argument("check_name") + parser.add_argument("kill_timeout", type=int) + parser.add_argument( + "--validate-bugfix", + action="store_true", + help="Check that added tests failed on latest stable", + ) + parser.add_argument( + "--post-commit-status", + default="commit_status", + choices=["commit_status", "file"], + help="Where to public post commit status", + ) + return parser.parse_args() + + +def docker_build_image(image_name: str, filepath: Path) -> DockerImage: + # context = filepath.parent + docker_image = DockerImage(image_name) + build_cmd = f"docker build --network=host -t {image_name} {filepath}" + logging.info("Will build image with cmd: '%s'", build_cmd) + subprocess.check_call( + build_cmd, + shell=True, + ) + return docker_image + + +def main(): + logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + + temp_path = TEMP_PATH + repo_path = REPO_COPY + reports_path = REPORTS_PATH + # post_commit_path = os.path.join(temp_path, "functional_commit_status.tsv") + + args = parse_args() + check_name = args.check_name + kill_timeout = args.kill_timeout + validate_bugfix_check = args.validate_bugfix + + # flaky_check = "flaky" in check_name.lower() + + # run_changed_tests = flaky_check or validate_bugfix_check + run_changed_tests = validate_bugfix_check + gh = Github(get_best_robot_token(), per_page=100) + + # For validate_bugfix_check we need up to date information about labels, so pr_event_from_api is used + pr_info = PRInfo( + need_changed_files=run_changed_tests, pr_event_from_api=validate_bugfix_check + ) + + commit = get_commit(gh, pr_info.sha) + atexit.register(update_mergeable_check, gh, pr_info, check_name) + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + + # if validate_bugfix_check and "pr-bugfix" not in pr_info.labels: + # if args.post_commit_status == "file": + # post_commit_status_to_file( + # post_commit_path, + # f"Skipped (no pr-bugfix in {pr_info.labels})", + # "success", + # "null", + # ) + # logging.info("Skipping '%s' (no pr-bugfix in %s)", check_name, pr_info.labels) + # sys.exit(0) + + if "RUN_BY_HASH_NUM" in os.environ: + run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM", "0")) + run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL", "0")) + check_name_with_group = ( + check_name + f" [{run_by_hash_num + 1}/{run_by_hash_total}]" + ) + else: + run_by_hash_num = 0 + run_by_hash_total = 0 + check_name_with_group = check_name + + rerun_helper = RerunHelper(commit, check_name_with_group) + if rerun_helper.is_already_finished_by_status(): + logging.info("Check is already finished according to github status, exiting") + sys.exit(0) + + # tests_to_run = [] + # if run_changed_tests: + # tests_to_run = get_tests_to_run(pr_info) + # if not tests_to_run: + # state = override_status("success", check_name, validate_bugfix_check) + # if args.post_commit_status == "commit_status": + # post_commit_status( + # commit, + # state, + # NotSet, + # NO_CHANGES_MSG, + # check_name_with_group, + # pr_info, + # ) + # elif args.post_commit_status == "file": + # post_commit_status_to_file( + # post_commit_path, + # description=NO_CHANGES_MSG, + # state=state, + # report_url="null", + # ) + # sys.exit(0) + + image_name = "clickhouse/libfuzzer-test" # get_image_name(check_name) + docker_image = docker_build_image( + image_name, Path("../../docker/test/libfuzzer/") + ) # get_image_with_version(reports_path, image_name) + + fuzzers_tmp_path = os.path.join(temp_path, "fuzzers_tmp") + if not os.path.exists(fuzzers_tmp_path): + os.makedirs(fuzzers_tmp_path) + + # if validate_bugfix_check: + # download_last_release(packages_path) + # else: + # download_all_deb_packages(check_name, reports_path, packages_path) + download_fuzzers(check_name, reports_path, fuzzers_tmp_path) + + fuzzers_path = os.path.join(temp_path, "fuzzers") + for fuzzer in os.listdir(fuzzers_tmp_path): + fuzzer_path = os.path.join(fuzzers_path, fuzzer) + os.makedirs(fuzzer_path) + os.rename( + os.path.join(fuzzers_tmp_path, fuzzer), os.path.join(fuzzer_path, fuzzer) + ) + + os.rmdir(fuzzers_tmp_path) + + # server_log_path = os.path.join(temp_path, "server_log") + # if not os.path.exists(server_log_path): + # os.makedirs(server_log_path) + + result_path = os.path.join(temp_path, "result_path") + if not os.path.exists(result_path): + os.makedirs(result_path) + + # run_log_path = os.path.join(result_path, "run.log") + + additional_envs = get_additional_envs( + check_name, run_by_hash_num, run_by_hash_total + ) + # if validate_bugfix_check: + # additional_envs.append("GLOBAL_TAGS=no-random-settings") + + ci_logs_credentials = CiLogsCredentials(Path(temp_path) / "export-logs-config.sh") + ci_logs_args = ci_logs_credentials.get_docker_arguments( + pr_info, stopwatch.start_time_str, check_name + ) + + run_command = get_run_command( + # check_name, + fuzzers_path, + repo_path, + result_path, + # server_log_path, + kill_timeout, + additional_envs, + ci_logs_args, + docker_image, + # flaky_check, + # tests_to_run, + ) + logging.info("Going to run func tests: %s", run_command) + + sys.exit(0) + + # with TeePopen(run_command, run_log_path) as process: + # retcode = process.wait() + # if retcode == 0: + # logging.info("Run successfully") + # else: + # logging.info("Run failed") + + # try: + # subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + # except subprocess.CalledProcessError: + # logging.warning("Failed to change files owner in %s, ignoring it", temp_path) + + # ci_logs_credentials.clean_ci_logs_from_credentials(Path(run_log_path)) + # s3_helper = S3Helper() + + # state, description, test_results, additional_logs = process_results( + # result_path, server_log_path + # ) + # state = override_status(state, check_name, invert=validate_bugfix_check) + + # ch_helper = ClickHouseHelper() + + # report_url = upload_results( + # s3_helper, + # pr_info.number, + # pr_info.sha, + # test_results, + # [run_log_path] + additional_logs, + # check_name_with_group, + # ) + + # print(f"::notice:: {check_name} Report url: {report_url}") + # if args.post_commit_status == "commit_status": + # post_commit_status( + # commit, state, report_url, description, check_name_with_group, pr_info + # ) + # elif args.post_commit_status == "file": + # post_commit_status_to_file( + # post_commit_path, + # description, + # state, + # report_url, + # ) + # else: + # raise Exception( + # f'Unknown post_commit_status option "{args.post_commit_status}"' + # ) + + # prepared_events = prepare_tests_results_for_clickhouse( + # pr_info, + # test_results, + # state, + # stopwatch.duration_seconds, + # stopwatch.start_time_str, + # report_url, + # check_name_with_group, + # ) + # ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) + + # if state != "success": + # if FORCE_TESTS_LABEL in pr_info.labels: + # print(f"'{FORCE_TESTS_LABEL}' enabled, will report success") + # else: + # sys.exit(1) + + +if __name__ == "__main__": + main() From 0847889db6e56f50d70e167915a69d4118b778b8 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 11 Sep 2023 19:06:00 +0000 Subject: [PATCH 36/93] libFuzzer infrastructure --- CMakeLists.txt | 1 + docker/packager/binary/build.sh | 1 + tests/ci/libfuzzer_test_check.py | 18 ++++-------------- tests/fuzz/build.sh | 28 ++++++++++++++++++++++++++++ 4 files changed, 34 insertions(+), 14 deletions(-) create mode 100755 tests/fuzz/build.sh diff --git a/CMakeLists.txt b/CMakeLists.txt index 781a9efe64a..b4e13e8ab5b 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -588,6 +588,7 @@ if (FUZZER) endif() endif() endforeach() + add_custom_command(TARGET fuzzers POST_BUILD COMMAND SRC=${CMAKE_SOURCE_DIR} BIN=${CMAKE_BINARY_DIR} OUT=${CMAKE_BINARY_DIR}/programs ${CMAKE_SOURCE_DIR}/tests/fuzz/build.sh VERBATIM) endif() include (cmake/sanitize_targets.cmake) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 75a18528e65..11efffd592c 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -100,6 +100,7 @@ fi mv ./programs/clickhouse* /output || mv ./programs/*_fuzzer /output [ -x ./programs/self-extracting/clickhouse ] && mv ./programs/self-extracting/clickhouse /output mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds +mv ./programs/*.dict ./programs/*.options ./programs/*_seed_corpus.zip /output ||: # libFuzzer oss-fuzz compatible infrastructure prepare_combined_output () { local OUTPUT diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 148b6e6d1e4..75af6ddf5d9 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -330,25 +330,15 @@ def main(): image_name, Path("../../docker/test/libfuzzer/") ) # get_image_with_version(reports_path, image_name) - fuzzers_tmp_path = os.path.join(temp_path, "fuzzers_tmp") - if not os.path.exists(fuzzers_tmp_path): - os.makedirs(fuzzers_tmp_path) + fuzzers_path = os.path.join(temp_path, "fuzzers") + if not os.path.exists(fuzzers_path): + os.makedirs(fuzzers_path) # if validate_bugfix_check: # download_last_release(packages_path) # else: # download_all_deb_packages(check_name, reports_path, packages_path) - download_fuzzers(check_name, reports_path, fuzzers_tmp_path) - - fuzzers_path = os.path.join(temp_path, "fuzzers") - for fuzzer in os.listdir(fuzzers_tmp_path): - fuzzer_path = os.path.join(fuzzers_path, fuzzer) - os.makedirs(fuzzer_path) - os.rename( - os.path.join(fuzzers_tmp_path, fuzzer), os.path.join(fuzzer_path, fuzzer) - ) - - os.rmdir(fuzzers_tmp_path) + download_fuzzers(check_name, reports_path, fuzzers_path) # server_log_path = os.path.join(temp_path, "server_log") # if not os.path.exists(server_log_path): diff --git a/tests/fuzz/build.sh b/tests/fuzz/build.sh new file mode 100755 index 00000000000..12f41f6e079 --- /dev/null +++ b/tests/fuzz/build.sh @@ -0,0 +1,28 @@ +#!/bin/bash -eu + +# copy fuzzer options and dictionaries +cp $SRC/tests/fuzz/*.dict $OUT/ +cp $SRC/tests/fuzz/*.options $OUT/ + +# prepare corpus dirs +mkdir -p $BIN/tests/fuzz/lexer_fuzzer.in/ +mkdir -p $BIN/tests/fuzz/select_parser_fuzzer.in/ +mkdir -p $BIN/tests/fuzz/create_parser_fuzzer.in/ +mkdir -p $BIN/tests/fuzz/execute_query_fuzzer.in/ + +# prepare corpus +cp $SRC/tests/queries/0_stateless/*.sql $BIN/tests/fuzz/lexer_fuzzer.in/ +cp $SRC/tests/queries/0_stateless/*.sql $BIN/tests/fuzz/select_parser_fuzzer.in/ +cp $SRC/tests/queries/0_stateless/*.sql $BIN/tests/fuzz/create_parser_fuzzer.in/ +cp $SRC/tests/queries/0_stateless/*.sql $BIN/tests/fuzz/execute_query_fuzzer.in/ +cp $SRC/tests/queries/1_stateful/*.sql $BIN/tests/fuzz/lexer_fuzzer.in/ +cp $SRC/tests/queries/1_stateful/*.sql $BIN/tests/fuzz/select_parser_fuzzer.in/ +cp $SRC/tests/queries/1_stateful/*.sql $BIN/tests/fuzz/create_parser_fuzzer.in/ +cp $SRC/tests/queries/1_stateful/*.sql $BIN/tests/fuzz/execute_query_fuzzer.in/ + +# build corpus archives +cd $BIN/tests/fuzz +for dir in *_fuzzer.in; do + fuzzer=$(basename $dir .in) + zip -rj "$OUT/${fuzzer}_seed_corpus.zip" "${dir}/" +done From eb3a7caa74748a5c9f97f3d740cefb16f9a4bae2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 11 Sep 2023 21:10:03 +0000 Subject: [PATCH 37/93] add zip to build docker --- docker/packager/binary/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 940daad9c61..12818335807 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -74,6 +74,7 @@ RUN add-apt-repository ppa:ubuntu-toolchain-r/test --yes \ python3-boto3 \ yasm \ zstd \ + zip \ && apt-get clean \ && rm -rf /var/lib/apt/lists From 44546458f09b913eb511eb1c332f06d0fad48a46 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 11 Sep 2023 22:45:50 +0000 Subject: [PATCH 38/93] add infrastructure files to the download filter --- tests/ci/build_download_helper.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 02e22e88a96..e27d10cbe5b 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -214,5 +214,8 @@ def download_performance_build(check_name, reports_path, result_path): def download_fuzzers(check_name, reports_path, result_path): download_builds_filter( - check_name, reports_path, result_path, lambda x: x.endswith("_fuzzer") + check_name, + reports_path, + result_path, + lambda x: x.endswith(("_fuzzer", ".dict", ".options", "_seed_corpus.zip")), ) From 1ad0a77c9f30146289bc640c1ba79de6db0d745e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 12 Sep 2023 16:30:52 +0000 Subject: [PATCH 39/93] unzip corpora --- tests/ci/libfuzzer_test_check.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 75af6ddf5d9..02f5c184b54 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -8,6 +8,7 @@ import re import subprocess import sys import atexit +import zipfile from pathlib import Path from typing import List, Tuple @@ -334,12 +335,15 @@ def main(): if not os.path.exists(fuzzers_path): os.makedirs(fuzzers_path) - # if validate_bugfix_check: - # download_last_release(packages_path) - # else: - # download_all_deb_packages(check_name, reports_path, packages_path) download_fuzzers(check_name, reports_path, fuzzers_path) + for file in os.listdir(fuzzers_path): + if file.endswith("_seed_corpus.zip"): + corpus_path = os.path.join( + temp_path, file.removesuffix("_seed_corpus.zip") + ".in" + ) + zipfile.ZipFile(os.path.join(temp_path, file), "r").extractall(corpus_path) + # server_log_path = os.path.join(temp_path, "server_log") # if not os.path.exists(server_log_path): # os.makedirs(server_log_path) @@ -374,7 +378,7 @@ def main(): # flaky_check, # tests_to_run, ) - logging.info("Going to run func tests: %s", run_command) + logging.info("Going to run libFuzzer tests: %s", run_command) sys.exit(0) From d80ae880606d2f40dae4dd9eb085a3016311a137 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 12 Sep 2023 20:56:42 +0000 Subject: [PATCH 40/93] run docker --- docker/test/libfuzzer/Dockerfile | 3 +- docker/test/libfuzzer/parse_options.py | 0 docker/test/libfuzzer/run_libfuzzer.sh | 142 +++++++++---------------- tests/ci/libfuzzer_test_check.py | 37 ++++--- 4 files changed, 77 insertions(+), 105 deletions(-) mode change 100644 => 100755 docker/test/libfuzzer/parse_options.py mode change 100644 => 100755 docker/test/libfuzzer/run_libfuzzer.sh diff --git a/docker/test/libfuzzer/Dockerfile b/docker/test/libfuzzer/Dockerfile index 77815431314..65cd8e4831f 100644 --- a/docker/test/libfuzzer/Dockerfile +++ b/docker/test/libfuzzer/Dockerfile @@ -33,9 +33,10 @@ RUN pip3 install Jinja2 COPY * / +ENV FUZZER_ARGS="-max_total_time=60" + SHELL ["/bin/bash", "-c"] CMD set -o pipefail \ - && cd /workspace \ && timeout -s 9 1h /run_libfuzzer.sh 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee main.log # docker run --network=host --volume :/workspace -e PR_TO_TEST=<> -e SHA_TO_TEST=<> clickhouse/libfuzzer diff --git a/docker/test/libfuzzer/parse_options.py b/docker/test/libfuzzer/parse_options.py old mode 100644 new mode 100755 diff --git a/docker/test/libfuzzer/run_libfuzzer.sh b/docker/test/libfuzzer/run_libfuzzer.sh old mode 100644 new mode 100755 index 49a59dafb90..b60e942f02a --- a/docker/test/libfuzzer/run_libfuzzer.sh +++ b/docker/test/libfuzzer/run_libfuzzer.sh @@ -1,115 +1,77 @@ #!/bin/bash -eu -# Copyright 2016 Google Inc. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -################################################################################ # Fuzzer runner. Appends .options arguments and seed corpus to users args. # Usage: $0 -export PATH=$OUT:$PATH -cd $OUT +# export PATH=$OUT:$PATH +# cd $OUT DEBUGGER=${DEBUGGER:-} +FUZZER_ARGS=${FUZZER_ARGS:-} -FUZZER=$1 -shift +function run_fuzzer() { + FUZZER=$1 -# This env var is set by CIFuzz. CIFuzz fills this directory with the corpus -# from ClusterFuzz. -CORPUS_DIR=${CORPUS_DIR:-} -if [ -z "$CORPUS_DIR" ] -then - CORPUS_DIR="/tmp/${FUZZER}_corpus" - rm -rf $CORPUS_DIR && mkdir -p $CORPUS_DIR -fi + echo Running fuzzer "$FUZZER" -SANITIZER=${SANITIZER:-} -if [ -z $SANITIZER ]; then - # If $SANITIZER is not specified (e.g. calling from `reproduce` command), it - # is not important and can be set to any value. - SANITIZER="default" -fi + CORPUS_DIR="" + if [ -d "${FUZZER}.in" ]; then + CORPUS_DIR="${FUZZER}.in" + fi -if [[ "$RUN_FUZZER_MODE" = interactive ]]; then - FUZZER_OUT="$OUT/${FUZZER}_${FUZZING_ENGINE}_${SANITIZER}_out" -else - FUZZER_OUT="/tmp/${FUZZER}_${FUZZING_ENGINE}_${SANITIZER}_out" -fi + OPTIONS_FILE="${FUZZER}.options" + CUSTOM_LIBFUZZER_OPTIONS="" + if [ -f "$OPTIONS_FILE" ]; then + custom_asan_options=$(/parse_options.py "$OPTIONS_FILE" asan) + if [ -n "$custom_asan_options" ]; then + export ASAN_OPTIONS="$ASAN_OPTIONS:$custom_asan_options" + fi -rm -rf $FUZZER_OUT && mkdir -p $FUZZER_OUT + custom_msan_options=$(/parse_options.py "$OPTIONS_FILE" msan) + if [ -n "$custom_msan_options" ]; then + export MSAN_OPTIONS="$MSAN_OPTIONS:$custom_msan_options" + fi -SEED_CORPUS="${FUZZER}_seed_corpus.zip" + custom_ubsan_options=$(/parse_options.py "$OPTIONS_FILE" ubsan) + if [ -n "$custom_ubsan_options" ]; then + export UBSAN_OPTIONS="$UBSAN_OPTIONS:$custom_ubsan_options" + fi -# TODO: Investigate why this code block is skipped -# by all default fuzzers in bad_build_check. -# They all set SKIP_SEED_CORPUS=1. -if [ -f $SEED_CORPUS ] && [ -z ${SKIP_SEED_CORPUS:-} ]; then - echo "Using seed corpus: $SEED_CORPUS" - unzip -o -d ${CORPUS_DIR}/ $SEED_CORPUS > /dev/null -fi + CUSTOM_LIBFUZZER_OPTIONS=$(/parse_options.py "$OPTIONS_FILE" libfuzzer) + fi -OPTIONS_FILE="${FUZZER}.options" -CUSTOM_LIBFUZZER_OPTIONS="" + CMD_LINE="./$FUZZER $FUZZER_ARGS" + CMD_LINE="$CMD_LINE $CORPUS_DIR" -if [ -f $OPTIONS_FILE ]; then - custom_asan_options=$(parse_options.py $OPTIONS_FILE asan) - if [ ! -z $custom_asan_options ]; then - export ASAN_OPTIONS="$ASAN_OPTIONS:$custom_asan_options" - fi + if [[ -n "$CUSTOM_LIBFUZZER_OPTIONS" ]]; then + CMD_LINE="$CMD_LINE $CUSTOM_LIBFUZZER_OPTIONS" + fi - custom_msan_options=$(parse_options.py $OPTIONS_FILE msan) - if [ ! -z $custom_msan_options ]; then - export MSAN_OPTIONS="$MSAN_OPTIONS:$custom_msan_options" - fi + if [[ ! "$CMD_LINE" =~ "-dict=" ]]; then + if [ -f "$FUZZER.dict" ]; then + CMD_LINE="$CMD_LINE -dict=$FUZZER.dict" + fi + fi - custom_ubsan_options=$(parse_options.py $OPTIONS_FILE ubsan) - if [ ! -z $custom_ubsan_options ]; then - export UBSAN_OPTIONS="$UBSAN_OPTIONS:$custom_ubsan_options" - fi + CMD_LINE="$CMD_LINE < /dev/null" - CUSTOM_LIBFUZZER_OPTIONS=$(parse_options.py $OPTIONS_FILE libfuzzer) -fi + echo "$CMD_LINE" + # Unset OUT so the fuzz target can't rely on it. + # unset OUT + if [ -n "$DEBUGGER" ]; then + CMD_LINE="$DEBUGGER $CMD_LINE" + fi -CMD_LINE="$OUT/$FUZZER $FUZZER_ARGS $*" + bash -c "$CMD_LINE" +} -if [ -z ${SKIP_SEED_CORPUS:-} ]; then -CMD_LINE="$CMD_LINE $CORPUS_DIR" -fi +ls -al -if [[ ! -z ${CUSTOM_LIBFUZZER_OPTIONS} ]]; then -CMD_LINE="$CMD_LINE $CUSTOM_LIBFUZZER_OPTIONS" -fi - -if [[ ! "$CMD_LINE" =~ "-dict=" ]]; then -if [ -f "$FUZZER.dict" ]; then - CMD_LINE="$CMD_LINE -dict=$FUZZER.dict" -fi -fi - -CMD_LINE="$CMD_LINE < /dev/null" - -echo $CMD_LINE - -# Unset OUT so the fuzz target can't rely on it. -unset OUT - -if [ ! -z "$DEBUGGER" ]; then - CMD_LINE="$DEBUGGER $CMD_LINE" -fi - -bash -c "$CMD_LINE" +for fuzzer in *_fuzzer; do + if [ -f "$fuzzer" ] && [ -x "$fuzzer" ]; then + run_fuzzer "$fuzzer" + fi +done diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 02f5c184b54..e7f907d02d4 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -41,7 +41,8 @@ from report import TestResults, read_test_results # from s3_helper import S3Helper from stopwatch import Stopwatch -# from tee_popen import TeePopen +from tee_popen import TeePopen + # from upload_result_helper import upload_results NO_CHANGES_MSG = "Nothing to run" @@ -121,8 +122,10 @@ def get_run_command( # ) return ( - f"docker run --volume={fuzzers_path}:/fuzzers " - f"{ci_logs_args}" + f"docker run " + f"{ci_logs_args} " + f"--workdir=/fuzzers " + f"--volume={fuzzers_path}:/fuzzers " f"--volume={repo_path}/tests:/usr/share/clickhouse-test " # f"{volume_with_broken_test}" f"--volume={result_path}:/test_output " @@ -338,11 +341,15 @@ def main(): download_fuzzers(check_name, reports_path, fuzzers_path) for file in os.listdir(fuzzers_path): - if file.endswith("_seed_corpus.zip"): + if file.endswith("_fuzzer"): + os.chmod(os.path.join(fuzzers_path, file), 0o777) + elif file.endswith("_seed_corpus.zip"): corpus_path = os.path.join( - temp_path, file.removesuffix("_seed_corpus.zip") + ".in" + fuzzers_path, file.removesuffix("_seed_corpus.zip") + ".in" + ) + zipfile.ZipFile(os.path.join(fuzzers_path, file), "r").extractall( + corpus_path ) - zipfile.ZipFile(os.path.join(temp_path, file), "r").extractall(corpus_path) # server_log_path = os.path.join(temp_path, "server_log") # if not os.path.exists(server_log_path): @@ -352,7 +359,7 @@ def main(): if not os.path.exists(result_path): os.makedirs(result_path) - # run_log_path = os.path.join(result_path, "run.log") + run_log_path = os.path.join(result_path, "run.log") additional_envs = get_additional_envs( check_name, run_by_hash_num, run_by_hash_total @@ -380,14 +387,16 @@ def main(): ) logging.info("Going to run libFuzzer tests: %s", run_command) - sys.exit(0) + # sys.exit(0) - # with TeePopen(run_command, run_log_path) as process: - # retcode = process.wait() - # if retcode == 0: - # logging.info("Run successfully") - # else: - # logging.info("Run failed") + with TeePopen(run_command, run_log_path) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + else: + logging.info("Run failed") + + sys.exit(0) # try: # subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) From cd0c775355e2eb2b620a638a1d2ce3c6f83c7f1c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 14 Sep 2023 03:10:55 +0000 Subject: [PATCH 41/93] review suggestions --- docker/images.json | 4 ++ docker/test/libfuzzer/Dockerfile | 2 +- docker/test/libfuzzer/run_libfuzzer.py | 73 ++++++++++++++++++++++++++ tests/ci/libfuzzer_test_check.py | 11 ++-- 4 files changed, 84 insertions(+), 6 deletions(-) create mode 100755 docker/test/libfuzzer/run_libfuzzer.py diff --git a/docker/images.json b/docker/images.json index d895e2da2f0..bddfd49ea3b 100644 --- a/docker/images.json +++ b/docker/images.json @@ -21,6 +21,10 @@ "name": "clickhouse/fuzzer", "dependent": [] }, + "docker/test/libfuzzer": { + "name": "clickhouse/libfuzzer", + "dependent": [] + }, "docker/test/performance-comparison": { "name": "clickhouse/performance-comparison", "dependent": [] diff --git a/docker/test/libfuzzer/Dockerfile b/docker/test/libfuzzer/Dockerfile index 65cd8e4831f..081cf5473f8 100644 --- a/docker/test/libfuzzer/Dockerfile +++ b/docker/test/libfuzzer/Dockerfile @@ -37,7 +37,7 @@ ENV FUZZER_ARGS="-max_total_time=60" SHELL ["/bin/bash", "-c"] CMD set -o pipefail \ - && timeout -s 9 1h /run_libfuzzer.sh 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee main.log + && timeout -s 9 1h /run_libfuzzer.py 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee main.log # docker run --network=host --volume :/workspace -e PR_TO_TEST=<> -e SHA_TO_TEST=<> clickhouse/libfuzzer diff --git a/docker/test/libfuzzer/run_libfuzzer.py b/docker/test/libfuzzer/run_libfuzzer.py new file mode 100755 index 00000000000..b608c97de60 --- /dev/null +++ b/docker/test/libfuzzer/run_libfuzzer.py @@ -0,0 +1,73 @@ +#!/usr/bin/env python3 + +import logging +import os +from pathlib import Path +import subprocess +from parse_options import parse_options + +DEBUGGER = os.getenv("DEBUGGER", "") +FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") + + +def run_fuzzer(fuzzer: str): + logging.info(f"Running fuzzer {fuzzer}...") + + corpus_dir = f"{fuzzer}.in" + with Path(corpus_dir) as path: + if not path.exists() or not path.is_dir(): + corpus_dir = "" + + options_file = f"{fuzzer}.options" + custom_libfuzzer_options = "" + + with Path(options_file) as path: + if path.exists() and path.is_file(): + custom_asan_options = parse_options(options_file, "asan") + if custom_asan_options: + os.environ[ + "ASAN_OPTIONS" + ] = f"{os.environ['ASAN_OPTIONS']}:{custom_asan_options}" + + custom_msan_options = parse_options(options_file, "msan") + if custom_msan_options: + os.environ[ + "MSAN_OPTIONS" + ] = f"{os.environ['MSAN_OPTIONS']}:{custom_msan_options}" + + custom_ubsan_options = parse_options(options_file, "ubsan") + if custom_ubsan_options: + os.environ[ + "UBSAN_OPTIONS" + ] = f"{os.environ['UBSAN_OPTIONS']}:{custom_ubsan_options}" + + custom_libfuzzer_options = parse_options(options_file, "libfuzzer") + + cmd_line = f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {corpus_dir}" + if custom_libfuzzer_options: + cmd_line += f" {custom_libfuzzer_options}" + + if not "-dict=" in cmd_line and Path(f"{fuzzer}.dict").exists(): + cmd_line += f" -dict={fuzzer}.dict" + + cmd_line += " < /dev/null" + + logging.info(f"...will execute: {cmd_line}") + subprocess.check_call(cmd_line, shell=True) + + +def main(): + logging.basicConfig(level=logging.INFO) + + subprocess.check_call("ls -al", shell=True) + + with Path() as current: + for fuzzer in current.iterdir(): + if (current / fuzzer).is_file() and os.access(current / fuzzer, os.X_OK): + run_fuzzer(fuzzer) + + exit(0) + + +if __name__ == "__main__": + main() diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index e7f907d02d4..41d08dade77 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -30,7 +30,7 @@ from commit_status_helper import ( # post_commit_status_to_file, update_mergeable_check, ) -from docker_pull_helper import DockerImage # , get_image_with_version +from docker_pull_helper import DockerImage, get_image_with_version # from download_release_packages import download_last_release from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH @@ -329,10 +329,11 @@ def main(): # ) # sys.exit(0) - image_name = "clickhouse/libfuzzer-test" # get_image_name(check_name) - docker_image = docker_build_image( - image_name, Path("../../docker/test/libfuzzer/") - ) # get_image_with_version(reports_path, image_name) + # image_name = "clickhouse/libfuzzer-test" # get_image_name(check_name) + # docker_image = docker_build_image( + # image_name, Path("../../docker/test/libfuzzer/") + # ) + docker_image = get_image_with_version(reports_path, "clickhouse/libfuzzer") fuzzers_path = os.path.join(temp_path, "fuzzers") if not os.path.exists(fuzzers_path): From 3a14bde95a54759cf8af6f1aac6d730dc2f3aad3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 14 Sep 2023 20:06:53 +0000 Subject: [PATCH 42/93] cleanup, fix tee to escape non-decodable symbols --- docker/test/libfuzzer/run_libfuzzer.sh | 77 ------- tests/ci/libfuzzer_test_check.py | 270 ++----------------------- tests/ci/tee_popen.py | 1 + 3 files changed, 17 insertions(+), 331 deletions(-) delete mode 100755 docker/test/libfuzzer/run_libfuzzer.sh diff --git a/docker/test/libfuzzer/run_libfuzzer.sh b/docker/test/libfuzzer/run_libfuzzer.sh deleted file mode 100755 index b60e942f02a..00000000000 --- a/docker/test/libfuzzer/run_libfuzzer.sh +++ /dev/null @@ -1,77 +0,0 @@ -#!/bin/bash -eu - -# Fuzzer runner. Appends .options arguments and seed corpus to users args. -# Usage: $0 - -# export PATH=$OUT:$PATH -# cd $OUT - -DEBUGGER=${DEBUGGER:-} -FUZZER_ARGS=${FUZZER_ARGS:-} - -function run_fuzzer() { - FUZZER=$1 - - echo Running fuzzer "$FUZZER" - - CORPUS_DIR="" - if [ -d "${FUZZER}.in" ]; then - CORPUS_DIR="${FUZZER}.in" - fi - - OPTIONS_FILE="${FUZZER}.options" - CUSTOM_LIBFUZZER_OPTIONS="" - - if [ -f "$OPTIONS_FILE" ]; then - custom_asan_options=$(/parse_options.py "$OPTIONS_FILE" asan) - if [ -n "$custom_asan_options" ]; then - export ASAN_OPTIONS="$ASAN_OPTIONS:$custom_asan_options" - fi - - custom_msan_options=$(/parse_options.py "$OPTIONS_FILE" msan) - if [ -n "$custom_msan_options" ]; then - export MSAN_OPTIONS="$MSAN_OPTIONS:$custom_msan_options" - fi - - custom_ubsan_options=$(/parse_options.py "$OPTIONS_FILE" ubsan) - if [ -n "$custom_ubsan_options" ]; then - export UBSAN_OPTIONS="$UBSAN_OPTIONS:$custom_ubsan_options" - fi - - CUSTOM_LIBFUZZER_OPTIONS=$(/parse_options.py "$OPTIONS_FILE" libfuzzer) - fi - - CMD_LINE="./$FUZZER $FUZZER_ARGS" - CMD_LINE="$CMD_LINE $CORPUS_DIR" - - if [[ -n "$CUSTOM_LIBFUZZER_OPTIONS" ]]; then - CMD_LINE="$CMD_LINE $CUSTOM_LIBFUZZER_OPTIONS" - fi - - if [[ ! "$CMD_LINE" =~ "-dict=" ]]; then - if [ -f "$FUZZER.dict" ]; then - CMD_LINE="$CMD_LINE -dict=$FUZZER.dict" - fi - fi - - CMD_LINE="$CMD_LINE < /dev/null" - - echo "$CMD_LINE" - - # Unset OUT so the fuzz target can't rely on it. - # unset OUT - - if [ -n "$DEBUGGER" ]; then - CMD_LINE="$DEBUGGER $CMD_LINE" - fi - - bash -c "$CMD_LINE" -} - -ls -al - -for fuzzer in *_fuzzer; do - if [ -f "$fuzzer" ] && [ -x "$fuzzer" ]; then - run_fuzzer "$fuzzer" - fi -done diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 41d08dade77..9fee997cc96 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -1,49 +1,37 @@ #!/usr/bin/env python3 import argparse -import csv import logging import os -import re import subprocess import sys import atexit import zipfile from pathlib import Path -from typing import List, Tuple +from typing import List from github import Github -# from build_download_helper import download_all_deb_packages from build_download_helper import download_fuzzers from clickhouse_helper import ( CiLogsCredentials, - # ClickHouseHelper, - # prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( - # NotSet, RerunHelper, get_commit, - # override_status, - # post_commit_status, - # post_commit_status_to_file, update_mergeable_check, ) from docker_pull_helper import DockerImage, get_image_with_version -# from download_release_packages import download_last_release from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH from get_robot_token import get_best_robot_token -from pr_info import PRInfo # , FORCE_TESTS_LABEL -from report import TestResults, read_test_results +from pr_info import PRInfo +from report import TestResults -# from s3_helper import S3Helper from stopwatch import Stopwatch from tee_popen import TeePopen -# from upload_result_helper import upload_results NO_CHANGES_MSG = "Nothing to run" @@ -70,34 +58,18 @@ def get_additional_envs(check_name, run_by_hash_num, run_by_hash_total): return result -# def get_image_name(check_name): -# if "stateless" in check_name.lower(): -# return "clickhouse/stateless-test" -# if "stateful" in check_name.lower(): -# return "clickhouse/stateful-test" -# else: -# raise Exception(f"Cannot deduce image name based on check name {check_name}") - - def get_run_command( - # check_name: str, - fuzzers_path: str, - repo_path: str, - result_path: str, - # server_log_path: str, + fuzzers_path: Path, + repo_path: Path, + result_path: Path, kill_timeout: int, additional_envs: List[str], ci_logs_args: str, image: DockerImage, - # flaky_check: bool, - # tests_to_run: List[str], ) -> str: additional_options = ["--hung-check"] additional_options.append("--print-time") - # if tests_to_run: - # additional_options += tests_to_run - additional_options_str = ( '-e ADDITIONAL_OPTIONS="' + " ".join(additional_options) + '"' ) @@ -108,18 +80,9 @@ def get_run_command( '-e S3_URL="https://s3.amazonaws.com/clickhouse-datasets"', ] - # if flaky_check: - # envs.append("-e NUM_TRIES=100") - # envs.append("-e MAX_RUN_TIME=1800") - envs += [f"-e {e}" for e in additional_envs] env_str = " ".join(envs) - # volume_with_broken_test = ( - # f"--volume={repo_path}/tests/analyzer_tech_debt.txt:/analyzer_tech_debt.txt " - # if "analyzer" in check_name - # else "" - # ) return ( f"docker run " @@ -127,96 +90,11 @@ def get_run_command( f"--workdir=/fuzzers " f"--volume={fuzzers_path}:/fuzzers " f"--volume={repo_path}/tests:/usr/share/clickhouse-test " - # f"{volume_with_broken_test}" f"--volume={result_path}:/test_output " - # f"--volume={server_log_path}:/var/log/clickhouse-server " f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image}" ) -def get_tests_to_run(pr_info: PRInfo) -> List[str]: - result = set() - - if pr_info.changed_files is None: - return [] - - for fpath in pr_info.changed_files: - if re.match(r"tests/queries/0_stateless/[0-9]{5}", fpath): - logging.info("File '%s' is changed and seems like a test", fpath) - fname = fpath.split("/")[3] - fname_without_ext = os.path.splitext(fname)[0] - # add '.' to the end of the test name not to run all tests with the same prefix - # e.g. we changed '00001_some_name.reference' - # and we have ['00001_some_name.sh', '00001_some_name_2.sql'] - # so we want to run only '00001_some_name.sh' - result.add(fname_without_ext + ".") - elif "tests/queries/" in fpath: - # log suspicious changes from tests/ for debugging in case of any problems - logging.info("File '%s' is changed, but it doesn't look like a test", fpath) - return list(result) - - -def process_results( - result_folder: str, - server_log_path: str, -) -> Tuple[str, str, TestResults, List[str]]: - test_results = [] # type: TestResults - additional_files = [] - # Just upload all files from result_folder. - # If task provides processed results, then it's responsible for content of result_folder. - if os.path.exists(result_folder): - test_files = [ - f - for f in os.listdir(result_folder) - if os.path.isfile(os.path.join(result_folder, f)) - ] - additional_files = [os.path.join(result_folder, f) for f in test_files] - - if os.path.exists(server_log_path): - server_log_files = [ - f - for f in os.listdir(server_log_path) - if os.path.isfile(os.path.join(server_log_path, f)) - ] - additional_files = additional_files + [ - os.path.join(server_log_path, f) for f in server_log_files - ] - - status = [] - status_path = os.path.join(result_folder, "check_status.tsv") - if os.path.exists(status_path): - logging.info("Found test_results.tsv") - with open(status_path, "r", encoding="utf-8") as status_file: - status = list(csv.reader(status_file, delimiter="\t")) - - if len(status) != 1 or len(status[0]) != 2: - logging.info("Files in result folder %s", os.listdir(result_folder)) - return "error", "Invalid check_status.tsv", test_results, additional_files - state, description = status[0][0], status[0][1] - - try: - results_path = Path(result_folder) / "test_results.tsv" - - if results_path.exists(): - logging.info("Found test_results.tsv") - else: - logging.info("Files in result folder %s", os.listdir(result_folder)) - return "error", "Not found test_results.tsv", test_results, additional_files - - test_results = read_test_results(results_path) - if len(test_results) == 0: - return "error", "Empty test_results.tsv", test_results, additional_files - except Exception as e: - return ( - "error", - f"Cannot parse test_results.tsv ({e})", - test_results, - additional_files, - ) - - return state, description, test_results, additional_files - - def parse_args(): parser = argparse.ArgumentParser() parser.add_argument("check_name") @@ -255,16 +133,12 @@ def main(): temp_path = TEMP_PATH repo_path = REPO_COPY reports_path = REPORTS_PATH - # post_commit_path = os.path.join(temp_path, "functional_commit_status.tsv") args = parse_args() check_name = args.check_name kill_timeout = args.kill_timeout validate_bugfix_check = args.validate_bugfix - # flaky_check = "flaky" in check_name.lower() - - # run_changed_tests = flaky_check or validate_bugfix_check run_changed_tests = validate_bugfix_check gh = Github(get_best_robot_token(), per_page=100) @@ -276,20 +150,9 @@ def main(): commit = get_commit(gh, pr_info.sha) atexit.register(update_mergeable_check, gh, pr_info, check_name) - if not os.path.exists(temp_path): + if not Path(temp_path).exists(): os.makedirs(temp_path) - # if validate_bugfix_check and "pr-bugfix" not in pr_info.labels: - # if args.post_commit_status == "file": - # post_commit_status_to_file( - # post_commit_path, - # f"Skipped (no pr-bugfix in {pr_info.labels})", - # "success", - # "null", - # ) - # logging.info("Skipping '%s' (no pr-bugfix in %s)", check_name, pr_info.labels) - # sys.exit(0) - if "RUN_BY_HASH_NUM" in os.environ: run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM", "0")) run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL", "0")) @@ -306,67 +169,30 @@ def main(): logging.info("Check is already finished according to github status, exiting") sys.exit(0) - # tests_to_run = [] - # if run_changed_tests: - # tests_to_run = get_tests_to_run(pr_info) - # if not tests_to_run: - # state = override_status("success", check_name, validate_bugfix_check) - # if args.post_commit_status == "commit_status": - # post_commit_status( - # commit, - # state, - # NotSet, - # NO_CHANGES_MSG, - # check_name_with_group, - # pr_info, - # ) - # elif args.post_commit_status == "file": - # post_commit_status_to_file( - # post_commit_path, - # description=NO_CHANGES_MSG, - # state=state, - # report_url="null", - # ) - # sys.exit(0) - - # image_name = "clickhouse/libfuzzer-test" # get_image_name(check_name) - # docker_image = docker_build_image( - # image_name, Path("../../docker/test/libfuzzer/") - # ) docker_image = get_image_with_version(reports_path, "clickhouse/libfuzzer") - fuzzers_path = os.path.join(temp_path, "fuzzers") - if not os.path.exists(fuzzers_path): + fuzzers_path = Path(temp_path) / "fuzzers" + if not fuzzers_path.exists(): os.makedirs(fuzzers_path) download_fuzzers(check_name, reports_path, fuzzers_path) for file in os.listdir(fuzzers_path): if file.endswith("_fuzzer"): - os.chmod(os.path.join(fuzzers_path, file), 0o777) + os.chmod(fuzzers_path / file, 0o777) elif file.endswith("_seed_corpus.zip"): - corpus_path = os.path.join( - fuzzers_path, file.removesuffix("_seed_corpus.zip") + ".in" - ) - zipfile.ZipFile(os.path.join(fuzzers_path, file), "r").extractall( - corpus_path - ) + corpus_path = fuzzers_path / (file.removesuffix("_seed_corpus.zip") + ".in") + zipfile.ZipFile(fuzzers_path / file, "r").extractall(corpus_path) - # server_log_path = os.path.join(temp_path, "server_log") - # if not os.path.exists(server_log_path): - # os.makedirs(server_log_path) - - result_path = os.path.join(temp_path, "result_path") - if not os.path.exists(result_path): + result_path = Path(temp_path) / "result_path" + if not result_path.exists(): os.makedirs(result_path) - run_log_path = os.path.join(result_path, "run.log") + run_log_path = result_path / "run.log" additional_envs = get_additional_envs( check_name, run_by_hash_num, run_by_hash_total ) - # if validate_bugfix_check: - # additional_envs.append("GLOBAL_TAGS=no-random-settings") ci_logs_credentials = CiLogsCredentials(Path(temp_path) / "export-logs-config.sh") ci_logs_args = ci_logs_credentials.get_docker_arguments( @@ -374,22 +200,16 @@ def main(): ) run_command = get_run_command( - # check_name, fuzzers_path, - repo_path, + Path(repo_path), result_path, - # server_log_path, kill_timeout, additional_envs, ci_logs_args, docker_image, - # flaky_check, - # tests_to_run, ) logging.info("Going to run libFuzzer tests: %s", run_command) - # sys.exit(0) - with TeePopen(run_command, run_log_path) as process: retcode = process.wait() if retcode == 0: @@ -399,64 +219,6 @@ def main(): sys.exit(0) - # try: - # subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) - # except subprocess.CalledProcessError: - # logging.warning("Failed to change files owner in %s, ignoring it", temp_path) - - # ci_logs_credentials.clean_ci_logs_from_credentials(Path(run_log_path)) - # s3_helper = S3Helper() - - # state, description, test_results, additional_logs = process_results( - # result_path, server_log_path - # ) - # state = override_status(state, check_name, invert=validate_bugfix_check) - - # ch_helper = ClickHouseHelper() - - # report_url = upload_results( - # s3_helper, - # pr_info.number, - # pr_info.sha, - # test_results, - # [run_log_path] + additional_logs, - # check_name_with_group, - # ) - - # print(f"::notice:: {check_name} Report url: {report_url}") - # if args.post_commit_status == "commit_status": - # post_commit_status( - # commit, state, report_url, description, check_name_with_group, pr_info - # ) - # elif args.post_commit_status == "file": - # post_commit_status_to_file( - # post_commit_path, - # description, - # state, - # report_url, - # ) - # else: - # raise Exception( - # f'Unknown post_commit_status option "{args.post_commit_status}"' - # ) - - # prepared_events = prepare_tests_results_for_clickhouse( - # pr_info, - # test_results, - # state, - # stopwatch.duration_seconds, - # stopwatch.start_time_str, - # report_url, - # check_name_with_group, - # ) - # ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) - - # if state != "success": - # if FORCE_TESTS_LABEL in pr_info.labels: - # print(f"'{FORCE_TESTS_LABEL}' enabled, will report success") - # else: - # sys.exit(1) - if __name__ == "__main__": main() diff --git a/tests/ci/tee_popen.py b/tests/ci/tee_popen.py index 7872b489951..a50532aea20 100644 --- a/tests/ci/tee_popen.py +++ b/tests/ci/tee_popen.py @@ -55,6 +55,7 @@ class TeePopen: stderr=STDOUT, stdout=PIPE, bufsize=1, + errors="backslashreplace", ) if self.timeout is not None and self.timeout > 0: t = Thread(target=self._check_timeout) From d1cd3cdd2a4dbbd7d695c09bdb09b7b6d1830400 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 16 Sep 2023 01:04:19 +0000 Subject: [PATCH 43/93] move on its own workflow --- .github/workflows/libfuzzer.yml | 80 +++++++++++++++++++++++++++++ .github/workflows/pull_request.yml | 82 ------------------------------ 2 files changed, 80 insertions(+), 82 deletions(-) create mode 100644 .github/workflows/libfuzzer.yml diff --git a/.github/workflows/libfuzzer.yml b/.github/workflows/libfuzzer.yml new file mode 100644 index 00000000000..74772ccf6d9 --- /dev/null +++ b/.github/workflows/libfuzzer.yml @@ -0,0 +1,80 @@ +name: libFuzzer + +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 + +on: # yamllint disable-line rule:truthy +# schedule: +# - cron: '0 0 2 31 1' # never for now + workflow_dispatch: +jobs: + BuilderFuzzers: + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=fuzzers + EOF + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + submodules: true + - name: Build + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v3 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + libFuzzerTest: + needs: [BuilderFuzzers] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/libfuzzer + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=libFuzzer tests + REPO_COPY=${{runner.temp}}/libfuzzer/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: libFuzzer test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 libfuzzer_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 7e56254bac0..ce135846dd5 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5059,88 +5059,6 @@ jobs: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" ############################################################################################# -#################################### libFuzzer build ######################################## -############################################################################################# - BuilderFuzzers: - if: contains(github.event.pull_request.labels.*.name, 'libFuzzer') - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=fuzzers - EOF - - name: Download changed images - uses: actions/download-artifact@v3 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - clear-repository: true - submodules: true - - name: Build - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v3 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" -############################################################################################## -################################ libFuzzer TESTS ############################################# -############################################################################################## - libFuzzerTest: - needs: [BuilderFuzzers] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/libfuzzer - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=libFuzzer tests - REPO_COPY=${{runner.temp}}/libfuzzer/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v3 - with: - path: ${{ env.REPORTS_PATH }} - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - clear-repository: true - - name: libFuzzer test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 libfuzzer_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################# ###################################### JEPSEN TESTS ######################################### ############################################################################################# Jepsen: From dd6f12dd94a93c916304ff9c0c0bd2dd2a40fcb9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 16 Sep 2023 04:41:13 +0000 Subject: [PATCH 44/93] review suggestions --- .github/workflows/libfuzzer.yml | 21 +++++++-- .github/workflows/pull_request.yml | 13 ++++-- docker/test/libfuzzer/parse_options.py | 61 -------------------------- docker/test/libfuzzer/run_libfuzzer.py | 26 ++++++----- tests/ci/libfuzzer_test_check.py | 42 +++--------------- 5 files changed, 48 insertions(+), 115 deletions(-) delete mode 100755 docker/test/libfuzzer/parse_options.py diff --git a/.github/workflows/libfuzzer.yml b/.github/workflows/libfuzzer.yml index 74772ccf6d9..e8a0396684a 100644 --- a/.github/workflows/libfuzzer.yml +++ b/.github/workflows/libfuzzer.yml @@ -5,9 +5,9 @@ env: PYTHONUNBUFFERED: 1 on: # yamllint disable-line rule:truthy -# schedule: -# - cron: '0 0 2 31 1' # never for now - workflow_dispatch: + # schedule: + # - cron: '0 0 2 31 1' # never for now + workflow_call: jobs: BuilderFuzzers: runs-on: [self-hosted, builder] @@ -21,11 +21,19 @@ jobs: CACHES_PATH=${{runner.temp}}/../ccaches BUILD_NAME=fuzzers EOF + - name: Download changed images + # even if artifact does not exist, e.g. on `do not test` label or failed Docker job + continue-on-error: true + uses: actions/download-artifact@v3 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true + ref: ${{github.ref}} - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -57,6 +65,13 @@ jobs: REPO_COPY=${{runner.temp}}/libfuzzer/ClickHouse KILL_TIMEOUT=10800 EOF + - name: Download changed images + # even if artifact does not exist, e.g. on `do not test` label or failed Docker job + continue-on-error: true + uses: actions/download-artifact@v3 + with: + name: changed_images + path: ${{ env.TEMP_PATH }} - name: Download json reports uses: actions/download-artifact@v3 with: diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index ce135846dd5..838a6b56440 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5186,9 +5186,16 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 finish_check.py python3 merge_pr.py --check-approved -############################################################################################## -########################### SQLLOGIC TEST ################################################### -############################################################################################## +############################################################################################# +####################################### libFuzzer ########################################### +############################################################################################# + libFuzzer: + if: contains(github.event.pull_request.labels.*.name, 'libFuzzer') + needs: [DockerHubPush, StyleCheck] + uses: ./.github/workflows/libfuzzer.yml + ############################################################################################## + ############################ SQLLOGIC TEST ################################################### + ############################################################################################## SQLLogicTestRelease: needs: [BuilderDebRelease] runs-on: [self-hosted, func-tester] diff --git a/docker/test/libfuzzer/parse_options.py b/docker/test/libfuzzer/parse_options.py deleted file mode 100755 index 5695e80a714..00000000000 --- a/docker/test/libfuzzer/parse_options.py +++ /dev/null @@ -1,61 +0,0 @@ -#!/usr/bin/env python3 -# Copyright 2020 Google LLC -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -################################################################################ -"""Helper script for parsing custom fuzzing options.""" -import configparser -import sys - - -def parse_options(options_file_path, options_section): - """Parses the given file and returns options from the given section.""" - parser = configparser.ConfigParser() - parser.read(options_file_path) - - if not parser.has_section(options_section): - return None - - options = parser[options_section] - - if options_section == "libfuzzer": - options_string = " ".join( - "-%s=%s" % (key, value) for key, value in options.items() - ) - else: - # Sanitizer options. - options_string = ":".join( - "%s=%s" % (key, value) for key, value in options.items() - ) - - return options_string - - -def main(): - """Processes the arguments and prints the options in the correct format.""" - if len(sys.argv) < 3: - sys.stderr.write( - "Usage: %s \n" % sys.argv[0] - ) - return 1 - - options = parse_options(sys.argv[1], sys.argv[2]) - if options is not None: - print(options) - - return 0 - - -if __name__ == "__main__": - sys.exit(main()) diff --git a/docker/test/libfuzzer/run_libfuzzer.py b/docker/test/libfuzzer/run_libfuzzer.py index b608c97de60..5ed019490d5 100755 --- a/docker/test/libfuzzer/run_libfuzzer.py +++ b/docker/test/libfuzzer/run_libfuzzer.py @@ -1,10 +1,10 @@ #!/usr/bin/env python3 +import configparser import logging import os from pathlib import Path import subprocess -from parse_options import parse_options DEBUGGER = os.getenv("DEBUGGER", "") FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") @@ -23,25 +23,29 @@ def run_fuzzer(fuzzer: str): with Path(options_file) as path: if path.exists() and path.is_file(): - custom_asan_options = parse_options(options_file, "asan") - if custom_asan_options: + parser = configparser.ConfigParser() + parser.read(path) + + if parser.has_section("asan"): os.environ[ "ASAN_OPTIONS" - ] = f"{os.environ['ASAN_OPTIONS']}:{custom_asan_options}" + ] = f"{os.environ['ASAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['asan'].items())}" - custom_msan_options = parse_options(options_file, "msan") - if custom_msan_options: + if parser.has_section("msan"): os.environ[ "MSAN_OPTIONS" - ] = f"{os.environ['MSAN_OPTIONS']}:{custom_msan_options}" + ] = f"{os.environ['MSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['msan'].items())}" - custom_ubsan_options = parse_options(options_file, "ubsan") - if custom_ubsan_options: + if parser.has_section("ubsan"): os.environ[ "UBSAN_OPTIONS" - ] = f"{os.environ['UBSAN_OPTIONS']}:{custom_ubsan_options}" + ] = f"{os.environ['UBSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['ubsan'].items())}" - custom_libfuzzer_options = parse_options(options_file, "libfuzzer") + if parser.has_section("libfuzzer"): + custom_libfuzzer_options = " ".join( + "-%s=%s" % (key, value) + for key, value in parser["libfuzzer"].items() + ) cmd_line = f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {corpus_dir}" if custom_libfuzzer_options: diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 9fee997cc96..8d307b22042 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -3,7 +3,6 @@ import argparse import logging import os -import subprocess import sys import atexit import zipfile @@ -99,59 +98,28 @@ def parse_args(): parser = argparse.ArgumentParser() parser.add_argument("check_name") parser.add_argument("kill_timeout", type=int) - parser.add_argument( - "--validate-bugfix", - action="store_true", - help="Check that added tests failed on latest stable", - ) - parser.add_argument( - "--post-commit-status", - default="commit_status", - choices=["commit_status", "file"], - help="Where to public post commit status", - ) return parser.parse_args() -def docker_build_image(image_name: str, filepath: Path) -> DockerImage: - # context = filepath.parent - docker_image = DockerImage(image_name) - build_cmd = f"docker build --network=host -t {image_name} {filepath}" - logging.info("Will build image with cmd: '%s'", build_cmd) - subprocess.check_call( - build_cmd, - shell=True, - ) - return docker_image - - def main(): logging.basicConfig(level=logging.INFO) stopwatch = Stopwatch() - temp_path = TEMP_PATH - repo_path = REPO_COPY + temp_path = Path(TEMP_PATH) + repo_path = Path(REPO_COPY) reports_path = REPORTS_PATH args = parse_args() check_name = args.check_name kill_timeout = args.kill_timeout - validate_bugfix_check = args.validate_bugfix - run_changed_tests = validate_bugfix_check gh = Github(get_best_robot_token(), per_page=100) - - # For validate_bugfix_check we need up to date information about labels, so pr_event_from_api is used - pr_info = PRInfo( - need_changed_files=run_changed_tests, pr_event_from_api=validate_bugfix_check - ) - + pr_info = PRInfo() commit = get_commit(gh, pr_info.sha) atexit.register(update_mergeable_check, gh, pr_info, check_name) - if not Path(temp_path).exists(): - os.makedirs(temp_path) + temp_path.mkdir(parents=True, exist_ok=True) if "RUN_BY_HASH_NUM" in os.environ: run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM", "0")) @@ -201,7 +169,7 @@ def main(): run_command = get_run_command( fuzzers_path, - Path(repo_path), + repo_path, result_path, kill_timeout, additional_envs, From a138d8a376a8656d35b3128673f982afb6ed966a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 17 Sep 2023 21:03:14 +0000 Subject: [PATCH 45/93] Use index to access shard addresses --- src/Interpreters/ClusterProxy/executeQuery.cpp | 18 +++++------------- 1 file changed, 5 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 4395f8373b2..9fb3f4f1b67 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -180,8 +180,10 @@ void executeQuery( ClusterPtr cluster = query_info.getCluster(); const size_t shards = cluster->getShardCount(); - for (const auto & shard_info : cluster->getShardsInfo()) + for (size_t i = 0, s = cluster->getShardsInfo().size(); i < s; ++i) { + const auto & shard_info = cluster->getShardsInfo()[i]; + ASTPtr query_ast_for_shard = query_ast->clone(); if (sharding_key_expr && query_info.optimized_cluster && settings.optimize_skip_unused_shards_rewrite_in && shards > 1) { @@ -213,18 +215,8 @@ void executeQuery( // decide for each shard if parallel reading from replicas should be enabled // according to settings and number of replicas declared per shard - bool parallel_replicas_enabled = false; - if (shard_info.shard_num > 0 && shard_info.shard_num <= cluster->getShardsAddresses().size()) - { - const auto & addresses = cluster->getShardsAddresses().at(shard_info.shard_num - 1); - parallel_replicas_enabled = addresses.size() > 1 && context->canUseParallelReplicas(); - } - else - { - chassert(shard_info.shard_num > 0); - - // FIXME or code: when can it happened (shard_num bigger than shard's addresses)? looks inconsistent - } + const auto & addresses = cluster->getShardsAddresses().at(i); + bool parallel_replicas_enabled = addresses.size() > 1 && context->canUseParallelReplicas(); stream_factory.createForShard(shard_info, query_ast_for_shard, main_table, table_func_ptr, From 66265e879dddee38080a9429a56ea0f4be3df291 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 18 Sep 2023 09:59:20 +0000 Subject: [PATCH 46/93] Make the transformation to one line reversible --- src/Parsers/tests/gtest_Parser.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index a20d6b2f111..a84f63d7cb1 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace { @@ -40,7 +41,7 @@ std::ostream & operator<<(std::ostream & ostr, const std::shared_ptr pa std::ostream & operator<<(std::ostream & ostr, const ParserTestCase & test_case) { std::string input_text{test_case.input_text}; - std::replace(input_text.begin(), input_text.end(),'\n', ' '); + boost::replace_all(input_text, "\n", "\\n"); return ostr << "ParserTestCase input: " << input_text; } From 2ea324e44e3a8580ad9ad5f16be53812e53017a3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 18 Sep 2023 09:59:30 +0000 Subject: [PATCH 47/93] Add explanatory comment --- src/Parsers/tests/gtest_Parser.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index a84f63d7cb1..3a90256f262 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -40,6 +40,8 @@ std::ostream & operator<<(std::ostream & ostr, const std::shared_ptr pa std::ostream & operator<<(std::ostream & ostr, const ParserTestCase & test_case) { + // New line characters are removed because at the time of writing this the unit test results are parsed from the + // command line output, and multi-line string representations are breaking the parsing logic. std::string input_text{test_case.input_text}; boost::replace_all(input_text, "\n", "\\n"); return ostr << "ParserTestCase input: " << input_text; From b14e00f9667bee0cabe63185b5e968d814fb73b8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Sep 2023 14:35:57 +0000 Subject: [PATCH 48/93] Docs: Update query cache section on non-deterministic functions Cf. https://github.com/ClickHouse/support-escalation/issues/963 --- docs/en/operations/query-cache.md | 20 +++++++++++++++++-- .../functions/date-time-functions.md | 8 ++++---- .../functions/ext-dict-functions.md | 2 +- .../functions/other-functions.md | 12 +++++------ .../functions/random-functions.md | 4 ++-- 5 files changed, 31 insertions(+), 15 deletions(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index d0b785d8fda..778767291f0 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -136,8 +136,24 @@ As a result, the query cache stores for each query multiple (partial) result blocks. While this behavior is a good default, it can be suppressed using setting [query_cache_squash_partial_query_results](settings/settings.md#query-cache-squash-partial-query-results). -Also, results of queries with non-deterministic functions such as `rand()` and `now()` are not cached. This can be overruled using -setting [query_cache_store_results_of_queries_with_nondeterministic_functions](settings/settings.md#query-cache-store-results-of-queries-with-nondeterministic-functions). +Also, results of queries with non-deterministic functions are not cached. Such functions include +- functions for accessing dictionaries: [`dictGet()`](../sql-reference/functions/ext-dict-functions.md#ext_dict_functions-dictGet) etc. +- [user-defined functions](sql-reference/statements/create/function.md), +- functions which return the current date or time: [`now()`](../sql-reference/functions/date-time-functions.md#date_time_functions-now), + [`today()`](../sql-reference/functions/date-time-functions.md#date_time_functions-today), + [`yesterday()`](../sql-reference/functions/date-time-functions.md#date_time_functions-yesterday) etc., +- functions which return random values: [`randomString()`](../sql-reference/functions/random-functions.md#random_functions-randomString), + [`fuzzBits()`](../sql-reference/functions/random-functions.md#random_functions-fuzzBits) etc., +- functions whose result depends on the size and order or the internal chunks used for query processing: + [`nowInBlock()`](../sql-reference/functions/date-time-functions.md#date_time_functions-nowInBlock) etc., + [`rowNumberInBlock()`](../sql-reference/functions/other-functions.md#other_functions-rowNumberInBlock), + [`runningDifference()`](../sql-reference/functions/other-functions.md#other_functions-runningDifference), + [`blockSize()`](../sql-reference/functions/other-functions.md#other_functions-blockSize) etc., +- functions which depend on the environment: [`currentUser()`](../sql-reference/functions/other-functions.md#other_functions-currentUser), + [`queryID()`](../sql-reference/functions/other-functions.md#other_functions-queryID), + [`getMacro()`](../sql-reference/functions/other-functions.md#other_functions-getMacro) etc. +Caching of non-determinstic functions can be forced regardless using setting +[query_cache_store_results_of_queries_with_nondeterministic_functions](settings/settings.md#query-cache-store-results-of-queries-with-nondeterministic-functions). Finally, entries in the query cache are not shared between users due to security reasons. For example, user A must not be able to bypass a row policy on a table by running the same query as another user B for whom no such policy exists. However, if necessary, cache entries can diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index af920ba2482..6297dd2dfe1 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1274,7 +1274,7 @@ Alias: `SUBDATE` **See Also** - [date_sub](#date_sub) -## now +## now {#date_time_functions-now} Returns the current date and time at the moment of query analysis. The function is a constant expression. @@ -1361,7 +1361,7 @@ Result: └─────────────────────────┴───────────────────────────────┘ ``` -## nowInBlock +## nowInBlock {#date_time_functions-nowInBlock} Returns the current date and time at the moment of processing of each block of data. In contrast to the function [now](#now), it is not a constant expression, and the returned value will be different in different blocks for long-running queries. @@ -1405,14 +1405,14 @@ Result: └─────────────────────┴─────────────────────┴──────────┘ ``` -## today +## today {#date_time_functions#today} Accepts zero arguments and returns the current date at one of the moments of query analysis. The same as ‘toDate(now())’. Aliases: `curdate`, `current_date`. -## yesterday +## yesterday {#date_time_functions#yesterday} Accepts zero arguments and returns yesterday’s date at one of the moments of query analysis. The same as ‘today() - 1’. diff --git a/docs/en/sql-reference/functions/ext-dict-functions.md b/docs/en/sql-reference/functions/ext-dict-functions.md index 284d6d80405..a61c15ba7e5 100644 --- a/docs/en/sql-reference/functions/ext-dict-functions.md +++ b/docs/en/sql-reference/functions/ext-dict-functions.md @@ -12,7 +12,7 @@ For dictionaries created with [DDL queries](../../sql-reference/statements/creat For information on connecting and configuring dictionaries, see [Dictionaries](../../sql-reference/dictionaries/index.md). -## dictGet, dictGetOrDefault, dictGetOrNull +## dictGet, dictGetOrDefault, dictGetOrNull {#ext_dict_functions-dictGet} Retrieves values from a dictionary. diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index bfbd26551d3..b8a3e3dde7e 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -11,7 +11,7 @@ sidebar_label: Other Returns the name of the host on which this function was executed. If the function executes on a remote server (distributed processing), the remote server name is returned. If the function executes in the context of a distributed table, it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. -## getMacro +## getMacro {#other_functions-getMacro} Returns a named value from the [macros](../../operations/server-configuration-parameters/settings.md#macros) section of the server configuration. @@ -186,7 +186,7 @@ Returns the type name of the passed argument. If `NULL` is passed, then the function returns type `Nullable(Nothing)`, which corresponds to ClickHouse's internal `NULL` representation. -## blockSize() +## blockSize() {#other_functions-blockSize} In ClickHouse, queries are processed in blocks (chunks). This function returns the size (row count) of the block the function is called on. @@ -311,7 +311,7 @@ Sleeps ‘seconds’ seconds for each row. The sleep time can be specified as in Returns the name of the current database. Useful in table engine parameters of `CREATE TABLE` queries where you need to specify the database. -## currentUser() +## currentUser() {#other_functions-currentUser} Returns the name of the current user. In case of a distributed query, the name of the user who initiated the query is returned. @@ -771,7 +771,7 @@ If executed in the context of a distributed table, this function generates a nor Returns the sequence number of the data block where the row is located. -## rowNumberInBlock() +## rowNumberInBlock() {#other_functions-rowNumberInBlock} Returns the ordinal number of the row in the data block. Different data blocks are always recalculated. @@ -896,7 +896,7 @@ Result: └────────────┴───────┴───────────┴────────────────┘ ``` -## runningDifference(x) +## runningDifference(x) {#other_functions-runningDifference} Calculates the difference between two consecutive row values in the data block. Returns 0 for the first row, and for subsequent rows the difference to the previous row. @@ -2274,7 +2274,7 @@ Result: └───────────────────────────┘ ``` -## queryID +## queryID {#other_functions-queryID} Returns the ID of the current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md index e593d9458f0..25b2bbec46c 100644 --- a/docs/en/sql-reference/functions/random-functions.md +++ b/docs/en/sql-reference/functions/random-functions.md @@ -478,7 +478,7 @@ Result: └─────────────────────┘ ``` -## randomString +## randomString {#random_functions#randomString} Generates a string of the specified length filled with random bytes (including zero bytes). Not all characters may be printable. @@ -627,7 +627,7 @@ Result: └──────────────────────┘ ``` -## fuzzBits +## fuzzBits {#random_functions-fuzzBits} **Syntax** From 6de45e355f4c90b752176c1a7701cc0da1eb6a0d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Sep 2023 15:27:38 +0000 Subject: [PATCH 49/93] Fix spelling/style --- docs/en/operations/query-cache.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 778767291f0..172303bf140 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -138,7 +138,7 @@ result blocks. While this behavior is a good default, it can be suppressed using Also, results of queries with non-deterministic functions are not cached. Such functions include - functions for accessing dictionaries: [`dictGet()`](../sql-reference/functions/ext-dict-functions.md#ext_dict_functions-dictGet) etc. -- [user-defined functions](sql-reference/statements/create/function.md), +- [user-defined functions](../sql-reference/statements/create/function.md), - functions which return the current date or time: [`now()`](../sql-reference/functions/date-time-functions.md#date_time_functions-now), [`today()`](../sql-reference/functions/date-time-functions.md#date_time_functions-today), [`yesterday()`](../sql-reference/functions/date-time-functions.md#date_time_functions-yesterday) etc., @@ -152,7 +152,7 @@ Also, results of queries with non-deterministic functions are not cached. Such f - functions which depend on the environment: [`currentUser()`](../sql-reference/functions/other-functions.md#other_functions-currentUser), [`queryID()`](../sql-reference/functions/other-functions.md#other_functions-queryID), [`getMacro()`](../sql-reference/functions/other-functions.md#other_functions-getMacro) etc. -Caching of non-determinstic functions can be forced regardless using setting +Caching of non-deterministic functions can be forced regardless using setting [query_cache_store_results_of_queries_with_nondeterministic_functions](settings/settings.md#query-cache-store-results-of-queries-with-nondeterministic-functions). Finally, entries in the query cache are not shared between users due to security reasons. For example, user A must not be able to bypass a From 326eaa9bc583bd0397c97760580679b5b50d46c6 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 18 Sep 2023 18:26:53 +0200 Subject: [PATCH 50/93] Better exception message in checkDataPart --- src/Storages/MergeTree/checkDataPart.cpp | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index a327ca17609..1717d91271a 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -123,9 +123,20 @@ static IMergeTreeDataPart::Checksums checkDataPart( if (data_part_storage.exists(IMergeTreeDataPart::SERIALIZATION_FILE_NAME)) { - auto serialization_file = data_part_storage.readFile(IMergeTreeDataPart::SERIALIZATION_FILE_NAME, read_settings, std::nullopt, std::nullopt); - SerializationInfo::Settings settings{ratio_of_defaults, false}; - serialization_infos = SerializationInfoByName::readJSON(columns_txt, settings, *serialization_file); + try + { + auto serialization_file = data_part_storage.readFile(IMergeTreeDataPart::SERIALIZATION_FILE_NAME, read_settings, std::nullopt, std::nullopt); + SerializationInfo::Settings settings{ratio_of_defaults, false}; + serialization_infos = SerializationInfoByName::readJSON(columns_txt, settings, *serialization_file); + } + catch (const Poco::Exception & ex) + { + throw Exception(ErrorCodes::CORRUPTED_DATA, "Failed to load {}, with error {}", IMergeTreeDataPart::SERIALIZATION_FILE_NAME, ex.message()); + } + catch (...) + { + throw; + } } auto get_serialization = [&serialization_infos](const auto & column) From 3753a78d1b558a71accac486298139402de7b909 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Sep 2023 17:24:49 +0000 Subject: [PATCH 51/93] Fix anchors --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- docs/en/sql-reference/functions/random-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 6297dd2dfe1..71b3d7c400b 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1405,7 +1405,7 @@ Result: └─────────────────────┴─────────────────────┴──────────┘ ``` -## today {#date_time_functions#today} +## today {#date_time_functions-today} Accepts zero arguments and returns the current date at one of the moments of query analysis. The same as ‘toDate(now())’. diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md index 25b2bbec46c..36e8d09f882 100644 --- a/docs/en/sql-reference/functions/random-functions.md +++ b/docs/en/sql-reference/functions/random-functions.md @@ -478,7 +478,7 @@ Result: └─────────────────────┘ ``` -## randomString {#random_functions#randomString} +## randomString {#random_functions-randomString} Generates a string of the specified length filled with random bytes (including zero bytes). Not all characters may be printable. From 6872e95fb2534db488b7fc5e76243e0a6c7fd754 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Sep 2023 17:34:40 +0000 Subject: [PATCH 52/93] Remove prefixes --- docs/en/operations/query-cache.md | 26 +++++++++---------- .../functions/date-time-functions.md | 8 +++--- .../functions/ext-dict-functions.md | 2 +- .../functions/other-functions.md | 12 ++++----- .../functions/random-functions.md | 4 +-- 5 files changed, 26 insertions(+), 26 deletions(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 172303bf140..137ebf8ad85 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -137,21 +137,21 @@ result blocks. While this behavior is a good default, it can be suppressed using [query_cache_squash_partial_query_results](settings/settings.md#query-cache-squash-partial-query-results). Also, results of queries with non-deterministic functions are not cached. Such functions include -- functions for accessing dictionaries: [`dictGet()`](../sql-reference/functions/ext-dict-functions.md#ext_dict_functions-dictGet) etc. +- functions for accessing dictionaries: [`dictGet()`](../sql-reference/functions/ext-dict-functions.md#dictGet) etc. - [user-defined functions](../sql-reference/statements/create/function.md), -- functions which return the current date or time: [`now()`](../sql-reference/functions/date-time-functions.md#date_time_functions-now), - [`today()`](../sql-reference/functions/date-time-functions.md#date_time_functions-today), - [`yesterday()`](../sql-reference/functions/date-time-functions.md#date_time_functions-yesterday) etc., -- functions which return random values: [`randomString()`](../sql-reference/functions/random-functions.md#random_functions-randomString), - [`fuzzBits()`](../sql-reference/functions/random-functions.md#random_functions-fuzzBits) etc., +- functions which return the current date or time: [`now()`](../sql-reference/functions/date-time-functions.md#now), + [`today()`](../sql-reference/functions/date-time-functions.md#today), + [`yesterday()`](../sql-reference/functions/date-time-functions.md#yesterday) etc., +- functions which return random values: [`randomString()`](../sql-reference/functions/random-functions.md#randomString), + [`fuzzBits()`](../sql-reference/functions/random-functions.md#fuzzBits) etc., - functions whose result depends on the size and order or the internal chunks used for query processing: - [`nowInBlock()`](../sql-reference/functions/date-time-functions.md#date_time_functions-nowInBlock) etc., - [`rowNumberInBlock()`](../sql-reference/functions/other-functions.md#other_functions-rowNumberInBlock), - [`runningDifference()`](../sql-reference/functions/other-functions.md#other_functions-runningDifference), - [`blockSize()`](../sql-reference/functions/other-functions.md#other_functions-blockSize) etc., -- functions which depend on the environment: [`currentUser()`](../sql-reference/functions/other-functions.md#other_functions-currentUser), - [`queryID()`](../sql-reference/functions/other-functions.md#other_functions-queryID), - [`getMacro()`](../sql-reference/functions/other-functions.md#other_functions-getMacro) etc. + [`nowInBlock()`](../sql-reference/functions/date-time-functions.md#nowInBlock) etc., + [`rowNumberInBlock()`](../sql-reference/functions/other-functions.md#rowNumberInBlock), + [`runningDifference()`](../sql-reference/functions/other-functions.md#runningDifference), + [`blockSize()`](../sql-reference/functions/other-functions.md#blockSize) etc., +- functions which depend on the environment: [`currentUser()`](../sql-reference/functions/other-functions.md#currentUser), + [`queryID()`](../sql-reference/functions/other-functions.md#queryID), + [`getMacro()`](../sql-reference/functions/other-functions.md#getMacro) etc. Caching of non-deterministic functions can be forced regardless using setting [query_cache_store_results_of_queries_with_nondeterministic_functions](settings/settings.md#query-cache-store-results-of-queries-with-nondeterministic-functions). diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 71b3d7c400b..0a065f087e1 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1274,7 +1274,7 @@ Alias: `SUBDATE` **See Also** - [date_sub](#date_sub) -## now {#date_time_functions-now} +## now {#now} Returns the current date and time at the moment of query analysis. The function is a constant expression. @@ -1361,7 +1361,7 @@ Result: └─────────────────────────┴───────────────────────────────┘ ``` -## nowInBlock {#date_time_functions-nowInBlock} +## nowInBlock {#nowInBlock} Returns the current date and time at the moment of processing of each block of data. In contrast to the function [now](#now), it is not a constant expression, and the returned value will be different in different blocks for long-running queries. @@ -1405,14 +1405,14 @@ Result: └─────────────────────┴─────────────────────┴──────────┘ ``` -## today {#date_time_functions-today} +## today {#today} Accepts zero arguments and returns the current date at one of the moments of query analysis. The same as ‘toDate(now())’. Aliases: `curdate`, `current_date`. -## yesterday {#date_time_functions#yesterday} +## yesterday {#yesterday} Accepts zero arguments and returns yesterday’s date at one of the moments of query analysis. The same as ‘today() - 1’. diff --git a/docs/en/sql-reference/functions/ext-dict-functions.md b/docs/en/sql-reference/functions/ext-dict-functions.md index a61c15ba7e5..4149afce044 100644 --- a/docs/en/sql-reference/functions/ext-dict-functions.md +++ b/docs/en/sql-reference/functions/ext-dict-functions.md @@ -12,7 +12,7 @@ For dictionaries created with [DDL queries](../../sql-reference/statements/creat For information on connecting and configuring dictionaries, see [Dictionaries](../../sql-reference/dictionaries/index.md). -## dictGet, dictGetOrDefault, dictGetOrNull {#ext_dict_functions-dictGet} +## dictGet, dictGetOrDefault, dictGetOrNull {#dictGet} Retrieves values from a dictionary. diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index b8a3e3dde7e..566eff76584 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -11,7 +11,7 @@ sidebar_label: Other Returns the name of the host on which this function was executed. If the function executes on a remote server (distributed processing), the remote server name is returned. If the function executes in the context of a distributed table, it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. -## getMacro {#other_functions-getMacro} +## getMacro {#getMacro} Returns a named value from the [macros](../../operations/server-configuration-parameters/settings.md#macros) section of the server configuration. @@ -186,7 +186,7 @@ Returns the type name of the passed argument. If `NULL` is passed, then the function returns type `Nullable(Nothing)`, which corresponds to ClickHouse's internal `NULL` representation. -## blockSize() {#other_functions-blockSize} +## blockSize() {#blockSize} In ClickHouse, queries are processed in blocks (chunks). This function returns the size (row count) of the block the function is called on. @@ -311,7 +311,7 @@ Sleeps ‘seconds’ seconds for each row. The sleep time can be specified as in Returns the name of the current database. Useful in table engine parameters of `CREATE TABLE` queries where you need to specify the database. -## currentUser() {#other_functions-currentUser} +## currentUser() {#currentUser} Returns the name of the current user. In case of a distributed query, the name of the user who initiated the query is returned. @@ -771,7 +771,7 @@ If executed in the context of a distributed table, this function generates a nor Returns the sequence number of the data block where the row is located. -## rowNumberInBlock() {#other_functions-rowNumberInBlock} +## rowNumberInBlock() {#rowNumberInBlock} Returns the ordinal number of the row in the data block. Different data blocks are always recalculated. @@ -896,7 +896,7 @@ Result: └────────────┴───────┴───────────┴────────────────┘ ``` -## runningDifference(x) {#other_functions-runningDifference} +## runningDifference(x) {#runningDifference} Calculates the difference between two consecutive row values in the data block. Returns 0 for the first row, and for subsequent rows the difference to the previous row. @@ -2274,7 +2274,7 @@ Result: └───────────────────────────┘ ``` -## queryID {#other_functions-queryID} +## queryID {#queryID} Returns the ID of the current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md index 36e8d09f882..13c29329f41 100644 --- a/docs/en/sql-reference/functions/random-functions.md +++ b/docs/en/sql-reference/functions/random-functions.md @@ -478,7 +478,7 @@ Result: └─────────────────────┘ ``` -## randomString {#random_functions-randomString} +## randomString {#randomString} Generates a string of the specified length filled with random bytes (including zero bytes). Not all characters may be printable. @@ -627,7 +627,7 @@ Result: └──────────────────────┘ ``` -## fuzzBits {#random_functions-fuzzBits} +## fuzzBits {#fuzzBits} **Syntax** From e8a292cbd406e1dc0260a481a87363da5b1f18bb Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Mon, 18 Sep 2023 11:08:41 -0700 Subject: [PATCH 53/93] Apply suggestions from code review Co-authored-by: Alexander Tokmakov --- docs/en/sql-reference/statements/system.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index f2494bfddde..8d7cdb80c2b 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -342,9 +342,11 @@ After running this statement the `[db.]replicated_merge_tree_family_table_name` ### SYNC DATABASE REPLICA -Sync all tables from all [replicated database](https://clickhouse.com/docs/en/engines/database-engines/replicated). This will make the ClickHouse synchronously fetch all table definition -from zookeeper and applied locally. +Waits until the specified [replicated database](https://clickhouse.com/docs/en/engines/database-engines/replicated) applies all schema changes from the DDL queue of that database. +**Syntax** +```sql +SYSTEM SYNC DATABASE REPLICA replicated_database_name; ### RESTART REPLICA Provides possibility to reinitialize Zookeeper session's state for `ReplicatedMergeTree` table, will compare current state with Zookeeper as source of truth and add tasks to Zookeeper queue if needed. From 4ed5b903b4a64f5f33644db1dee02e32c062e391 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Sep 2023 18:29:13 +0000 Subject: [PATCH 54/93] Docs: remove anchor prefix --- .../integrations/embedded-rocksdb.md | 4 +- .../table-engines/integrations/kafka.md | 2 +- .../table-engines/integrations/nats.md | 2 +- .../table-engines/integrations/rabbitmq.md | 2 +- .../table-engines/special/keepermap.md | 4 +- .../settings.md | 76 ++++---- docs/en/operations/settings/settings.md | 166 +++++++++--------- docs/en/sql-reference/dictionaries/index.md | 2 +- .../functions/array-functions.md | 6 +- .../functions/date-time-functions.md | 6 +- .../sql-reference/functions/hash-functions.md | 34 ++-- 11 files changed, 152 insertions(+), 152 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 2de981d33b7..23ab89e1983 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -8,7 +8,7 @@ sidebar_label: EmbeddedRocksDB This engine allows integrating ClickHouse with [rocksdb](http://rocksdb.org/). -## Creating a Table {#table_engine-EmbeddedRocksDB-creating-a-table} +## Creating a Table {#creating-a-table} ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] @@ -85,7 +85,7 @@ You can also change any [rocksdb options](https://github.com/facebook/rocksdb/wi ``` -## Supported operations {#table_engine-EmbeddedRocksDB-supported-operations} +## Supported operations {#supported-operations} ### Inserts diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index e4d3ac762ed..5d04dce4c51 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -14,7 +14,7 @@ Kafka lets you: - Organize fault-tolerant storage. - Process streams as they become available. -## Creating a Table {#table_engine-kafka-creating-a-table} +## Creating a Table {#creating-a-table} ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] diff --git a/docs/en/engines/table-engines/integrations/nats.md b/docs/en/engines/table-engines/integrations/nats.md index 25f442e5ce7..570b219e5fa 100644 --- a/docs/en/engines/table-engines/integrations/nats.md +++ b/docs/en/engines/table-engines/integrations/nats.md @@ -13,7 +13,7 @@ This engine allows integrating ClickHouse with [NATS](https://nats.io/). - Publish or subscribe to message subjects. - Process new messages as they become available. -## Creating a Table {#table_engine-redisstreams-creating-a-table} +## Creating a Table {#creating-a-table} ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index 3fd5a130173..4f6107764ec 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -13,7 +13,7 @@ This engine allows integrating ClickHouse with [RabbitMQ](https://www.rabbitmq.c - Publish or subscribe to data flows. - Process streams as they become available. -## Creating a Table {#table_engine-rabbitmq-creating-a-table} +## Creating a Table {#creating-a-table} ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] diff --git a/docs/en/engines/table-engines/special/keepermap.md b/docs/en/engines/table-engines/special/keepermap.md index 6ce56adbae9..5559cc2c648 100644 --- a/docs/en/engines/table-engines/special/keepermap.md +++ b/docs/en/engines/table-engines/special/keepermap.md @@ -20,7 +20,7 @@ For example: where path can be any other valid ZooKeeper path. -## Creating a Table {#table_engine-KeeperMap-creating-a-table} +## Creating a Table {#creating-a-table} ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] @@ -74,7 +74,7 @@ If multiple tables are created on the same ZooKeeper path, the values are persis As a result, it is possible to use `ON CLUSTER` clause when creating the table and sharing the data from multiple ClickHouse instances. Of course, it's possible to manually run `CREATE TABLE` with same path on unrelated ClickHouse instances to have same data sharing effect. -## Supported operations {#table_engine-KeeperMap-supported-operations} +## Supported operations {#supported-operations} ### Inserts diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index b27586e705f..5586dbafd88 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -835,7 +835,7 @@ List of prefixes for [custom settings](../../operations/settings/index.md#custom - [Custom settings](../../operations/settings/index.md#custom_settings) -## core_dump {#server_configuration_parameters-core_dump} +## core_dump {#core_dump} Configures soft limit for core dump file size. @@ -924,7 +924,7 @@ The path to the table in ZooKeeper. {replica} ``` -## dictionaries_config {#server_configuration_parameters-dictionaries_config} +## dictionaries_config {#dictionaries_config} The path to the config file for dictionaries. @@ -941,7 +941,7 @@ See also “[Dictionaries](../../sql-reference/dictionaries/index.md)”. *_dictionary.xml ``` -## user_defined_executable_functions_config {#server_configuration_parameters-user_defined_executable_functions_config} +## user_defined_executable_functions_config {#user_defined_executable_functions_config} The path to the config file for executable user defined functions. @@ -958,7 +958,7 @@ See also “[Executable User Defined Functions](../../sql-reference/functions/in *_function.xml ``` -## dictionaries_lazy_load {#server_configuration_parameters-dictionaries_lazy_load} +## dictionaries_lazy_load {#dictionaries_lazy_load} Lazy loading of dictionaries. @@ -974,7 +974,7 @@ The default is `true`. true ``` -## format_schema_path {#server_configuration_parameters-format_schema_path} +## format_schema_path {#format_schema_path} The path to the directory with the schemes for the input data, such as schemas for the [CapnProto](../../interfaces/formats.md#capnproto) format. @@ -985,7 +985,7 @@ The path to the directory with the schemes for the input data, such as schemas f format_schemas/ ``` -## graphite {#server_configuration_parameters-graphite} +## graphite {#graphite} Sending data to [Graphite](https://github.com/graphite-project). @@ -1019,7 +1019,7 @@ You can configure multiple `` clauses. For instance, you can use this ``` -## graphite_rollup {#server_configuration_parameters-graphite-rollup} +## graphite_rollup {#graphite-rollup} Settings for thinning data for Graphite. @@ -1051,7 +1051,7 @@ For more details, see [GraphiteMergeTree](../../engines/table-engines/mergetree- The port for connecting to the server over HTTP(s). -If `https_port` is specified, [openSSL](#server_configuration_parameters-openssl) must be configured. +If `https_port` is specified, [openSSL](#openssl) must be configured. If `http_port` is specified, the OpenSSL configuration is ignored even if it is set. @@ -1061,7 +1061,7 @@ If `http_port` is specified, the OpenSSL configuration is ignored even if it is 9999 ``` -## http_server_default_response {#server_configuration_parameters-http_server_default_response} +## http_server_default_response {#http_server_default_response} The page that is shown by default when you access the ClickHouse HTTP(s) server. The default value is “Ok.” (with a line feed at the end) @@ -1086,7 +1086,7 @@ Expired time for HSTS in seconds. The default value is 0 means clickhouse disabl 600000 ``` -## include_from {#server_configuration_parameters-include_from} +## include_from {#include_from} The path to the file with substitutions. @@ -1222,7 +1222,7 @@ The number of seconds that ClickHouse waits for incoming requests before closing 10 ``` -## listen_host {#server_configuration_parameters-listen_host} +## listen_host {#listen_host} Restriction on hosts that requests can come from. If you want the server to answer all of them, specify `::`. @@ -1233,7 +1233,7 @@ Examples: 127.0.0.1 ``` -## listen_backlog {#server_configuration_parameters-listen_backlog} +## listen_backlog {#listen_backlog} Backlog (queue size of pending connections) of the listen socket. @@ -1253,7 +1253,7 @@ Examples: 4096 ``` -## logger {#server_configuration_parameters-logger} +## logger {#logger} Logging settings. @@ -1357,7 +1357,7 @@ Keys for syslog: Default value: `LOG_USER` if `address` is specified, `LOG_DAEMON` otherwise. - format – Message format. Possible values: `bsd` and `syslog.` -## send_crash_reports {#server_configuration_parameters-send_crash_reports} +## send_crash_reports {#send_crash_reports} Settings for opt-in sending crash reports to the ClickHouse core developers team via [Sentry](https://sentry.io). Enabling it, especially in pre-production environments, is highly appreciated. @@ -1629,7 +1629,7 @@ Default value: `0.5`. -## merge_tree {#server_configuration_parameters-merge_tree} +## merge_tree {#merge_tree} Fine tuning for tables in the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). @@ -1676,7 +1676,7 @@ To disable `metric_log` setting, you should create the following file `/etc/clic ``` -## replicated_merge_tree {#server_configuration_parameters-replicated_merge_tree} +## replicated_merge_tree {#replicated_merge_tree} Fine tuning for tables in the [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/mergetree.md). @@ -1692,7 +1692,7 @@ For more information, see the MergeTreeSettings.h header file. ``` -## openSSL {#server_configuration_parameters-openssl} +## openSSL {#openssl} SSL client/server configuration. @@ -1751,7 +1751,7 @@ Keys for server/client settings: ``` -## part_log {#server_configuration_parameters-part-log} +## part_log {#part-log} Logging events that are associated with [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). For instance, adding or merging data. You can use the log to simulate merge algorithms and compare their characteristics. You can visualize the merge process. @@ -1791,7 +1791,7 @@ Default: false. ``` -## path {#server_configuration_parameters-path} +## path {#path} The path to the directory containing data. @@ -1805,7 +1805,7 @@ The trailing slash is mandatory. /var/lib/clickhouse/ ``` -## Prometheus {#server_configuration_parameters-prometheus} +## Prometheus {#prometheus} Exposing metrics data for scraping from [Prometheus](https://prometheus.io). @@ -1841,7 +1841,7 @@ Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse serve curl 127.0.0.1:9363/metrics ``` -## query_log {#server_configuration_parameters-query-log} +## query_log {#query-log} Setting for logging queries received with the [log_queries=1](../../operations/settings/settings.md) setting. @@ -1911,7 +1911,7 @@ Data for the query cache is allocated in DRAM. If memory is scarce, make sure to ``` -## query_thread_log {#server_configuration_parameters-query_thread_log} +## query_thread_log {#query_thread_log} Setting for logging threads of queries received with the [log_query_threads=1](../../operations/settings/settings.md#settings-log-query-threads) setting. @@ -1953,7 +1953,7 @@ If the table does not exist, ClickHouse will create it. If the structure of the ``` -## query_views_log {#server_configuration_parameters-query_views_log} +## query_views_log {#query_views_log} Setting for logging views (live, materialized etc) dependant of queries received with the [log_query_views=1](../../operations/settings/settings.md#settings-log-query-views) setting. @@ -1995,7 +1995,7 @@ If the table does not exist, ClickHouse will create it. If the structure of the ``` -## text_log {#server_configuration_parameters-text_log} +## text_log {#text_log} Settings for the [text_log](../../operations/system-tables/text_log.md#system_tables-text_log) system table for logging text messages. @@ -2037,7 +2037,7 @@ Default: false. ``` -## trace_log {#server_configuration_parameters-trace_log} +## trace_log {#trace_log} Settings for the [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table operation. @@ -2073,7 +2073,7 @@ The default server configuration file `config.xml` contains the following settin ``` -## asynchronous_insert_log {#server_configuration_parameters-asynchronous_insert_log} +## asynchronous_insert_log {#asynchronous_insert_log} Settings for the [asynchronous_insert_log](../../operations/system-tables/asynchronous_insert_log.md#system_tables-asynchronous_insert_log) system table for logging async inserts. @@ -2112,7 +2112,7 @@ Default: false. ``` -## crash_log {#server_configuration_parameters-crash_log} +## crash_log {#crash_log} Settings for the [crash_log](../../operations/system-tables/crash-log.md) system table operation. @@ -2150,7 +2150,7 @@ The default server configuration file `config.xml` contains the following settin ``` -## backup_log {#server_configuration_parameters-backup_log} +## backup_log {#backup_log} Settings for the [backup_log](../../operations/system-tables/backup_log.md) system table for logging `BACKUP` and `RESTORE` operations. @@ -2239,7 +2239,7 @@ For the value of the `incl` attribute, see the section “[Configuration files]( - [Cluster Discovery](../../operations/cluster-discovery.md) - [Replicated database engine](../../engines/database-engines/replicated.md) -## timezone {#server_configuration_parameters-timezone} +## timezone {#timezone} The server’s time zone. @@ -2257,7 +2257,7 @@ The time zone is necessary for conversions between String and DateTime formats w - [session_timezone](../settings/settings.md#session_timezone) -## tcp_port {#server_configuration_parameters-tcp_port} +## tcp_port {#tcp_port} Port for communicating with clients over the TCP protocol. @@ -2267,9 +2267,9 @@ Port for communicating with clients over the TCP protocol. 9000 ``` -## tcp_port_secure {#server_configuration_parameters-tcp_port_secure} +## tcp_port_secure {#tcp_port_secure} -TCP port for secure communication with clients. Use it with [OpenSSL](#server_configuration_parameters-openssl) settings. +TCP port for secure communication with clients. Use it with [OpenSSL](#openssl) settings. **Possible values** @@ -2281,7 +2281,7 @@ Positive integer. 9440 ``` -## mysql_port {#server_configuration_parameters-mysql_port} +## mysql_port {#mysql_port} Port for communicating with clients over MySQL protocol. @@ -2295,7 +2295,7 @@ Example 9004 ``` -## postgresql_port {#server_configuration_parameters-postgresql_port} +## postgresql_port {#postgresql_port} Port for communicating with clients over PostgreSQL protocol. @@ -2326,7 +2326,7 @@ Path on the local filesystem to store temporary data for processing large querie ``` -## user_files_path {#server_configuration_parameters-user_files_path} +## user_files_path {#user_files_path} The directory with user files. Used in the table function [file()](../../sql-reference/table-functions/file.md). @@ -2336,7 +2336,7 @@ The directory with user files. Used in the table function [file()](../../sql-ref /var/lib/clickhouse/user_files/ ``` -## user_scripts_path {#server_configuration_parameters-user_scripts_path} +## user_scripts_path {#user_scripts_path} The directory with user scripts files. Used for Executable user defined functions [Executable User Defined Functions](../../sql-reference/functions/index.md#executable-user-defined-functions). @@ -2346,7 +2346,7 @@ The directory with user scripts files. Used for Executable user defined function /var/lib/clickhouse/user_scripts/ ``` -## user_defined_path {#server_configuration_parameters-user_defined_path} +## user_defined_path {#user_defined_path} The directory with user defined files. Used for SQL user defined functions [SQL User Defined Functions](../../sql-reference/functions/index.md#user-defined-functions). @@ -2442,7 +2442,7 @@ Storage method for data part headers in ZooKeeper. This setting only applies to the `MergeTree` family. It can be specified: -- Globally in the [merge_tree](#server_configuration_parameters-merge_tree) section of the `config.xml` file. +- Globally in the [merge_tree](#merge_tree) section of the `config.xml` file. ClickHouse uses the setting for all the tables on the server. You can change the setting at any time. Existing tables change their behaviour when the setting changes. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e9e5920fa59..4282d5d6397 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -177,7 +177,7 @@ If `enable_optimize_predicate_expression = 1`, then the execution time of these If `enable_optimize_predicate_expression = 0`, then the execution time of the second query is much longer because the `WHERE` clause applies to all the data after the subquery finishes. -## fallback_to_stale_replicas_for_distributed_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} +## fallback_to_stale_replicas_for_distributed_queries {#fallback_to_stale_replicas_for_distributed_queries} Forces a query to an out-of-date replica if updated data is not available. See [Replication](../../engines/table-engines/mergetree-family/replication.md). @@ -187,7 +187,7 @@ Used when performing `SELECT` from a distributed table that points to replicated By default, 1 (enabled). -## force_index_by_date {#settings-force_index_by_date} +## force_index_by_date {#force_index_by_date} Disables query execution if the index can’t be used by date. @@ -203,7 +203,7 @@ Works with tables in the MergeTree family. If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition reduces the amount of data to read. For more information about data ranges in MergeTree tables, see [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -## use_skip_indexes {#settings-use_skip_indexes} +## use_skip_indexes {#use_skip_indexes} Use data skipping indexes during query execution. @@ -214,7 +214,7 @@ Possible values: Default value: 1. -## force_data_skipping_indices {#settings-force_data_skipping_indices} +## force_data_skipping_indices {#force_data_skipping_indices} Disables query execution if passed data skipping indices wasn't used. @@ -241,7 +241,7 @@ SELECT * FROM data_01515 WHERE d1 = 0 SETTINGS force_data_skipping_indices='`d1_ SELECT * FROM data_01515 WHERE d1 = 0 AND assumeNotNull(d1_null) = 0 SETTINGS force_data_skipping_indices='`d1_idx`, d1_null_idx'; -- Ok. ``` -## ignore_data_skipping_indices {#settings-ignore_data_skipping_indices} +## ignore_data_skipping_indices {#ignore_data_skipping_indices} Ignores the skipping indexes specified if used by the query. @@ -401,7 +401,7 @@ Enables or disables [fsync](http://pubs.opengroup.org/onlinepubs/9699919799/func It makes sense to disable it if the server has millions of tiny tables that are constantly being created and destroyed. -## function_range_max_elements_in_block {#settings-function_range_max_elements_in_block} +## function_range_max_elements_in_block {#function_range_max_elements_in_block} Sets the safety threshold for data volume generated by function [range](../../sql-reference/functions/array-functions.md/#range). Defines the maximum number of values generated by function per block of data (sum of array sizes for every row in a block). @@ -416,7 +416,7 @@ Default value: `500,000,000`. - [max_block_size](#setting-max_block_size) - [min_insert_block_size_rows](#min-insert-block-size-rows) -## enable_http_compression {#settings-enable_http_compression} +## enable_http_compression {#enable_http_compression} Enables or disables data compression in the response to an HTTP request. @@ -429,15 +429,15 @@ Possible values: Default value: 0. -## http_zlib_compression_level {#settings-http_zlib_compression_level} +## http_zlib_compression_level {#http_zlib_compression_level} -Sets the level of data compression in the response to an HTTP request if [enable_http_compression = 1](#settings-enable_http_compression). +Sets the level of data compression in the response to an HTTP request if [enable_http_compression = 1](#enable_http_compression). Possible values: Numbers from 1 to 9. Default value: 3. -## http_native_compression_disable_checksumming_on_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} +## http_native_compression_disable_checksumming_on_decompress {#http_native_compression_disable_checksumming_on_decompress} Enables or disables checksum verification when decompressing the HTTP POST data from the client. Used only for ClickHouse native compression format (not used with `gzip` or `deflate`). @@ -480,7 +480,7 @@ Possible values: Default value: `1000`. -## send_progress_in_http_headers {#settings-send_progress_in_http_headers} +## send_progress_in_http_headers {#send_progress_in_http_headers} Enables or disables `X-ClickHouse-Progress` HTTP response headers in `clickhouse-server` responses. @@ -518,7 +518,7 @@ Possible values: Default value: `1`. -## join_default_strictness {#settings-join_default_strictness} +## join_default_strictness {#join_default_strictness} Sets default strictness for [JOIN clauses](../../sql-reference/statements/select/join.md/#select-join). @@ -531,7 +531,7 @@ Possible values: Default value: `ALL`. -## join_algorithm {#settings-join_algorithm} +## join_algorithm {#join_algorithm} Specifies which [JOIN](../../sql-reference/statements/select/join.md) algorithm is used. @@ -547,7 +547,7 @@ Possible values: [Grace hash join](https://en.wikipedia.org/wiki/Hash_join#Grace_hash_join) is used. Grace hash provides an algorithm option that provides performant complex joins while limiting memory use. - The first phase of a grace join reads the right table and splits it into N buckets depending on the hash value of key columns (initially, N is `grace_hash_join_initial_buckets`). This is done in a way to ensure that each bucket can be processed independently. Rows from the first bucket are added to an in-memory hash table while the others are saved to disk. If the hash table grows beyond the memory limit (e.g., as set by [`max_bytes_in_join`](/docs/en/operations/settings/query-complexity.md/#settings-max_bytes_in_join)), the number of buckets is increased and the assigned bucket for each row. Any rows which don’t belong to the current bucket are flushed and reassigned. + The first phase of a grace join reads the right table and splits it into N buckets depending on the hash value of key columns (initially, N is `grace_hash_join_initial_buckets`). This is done in a way to ensure that each bucket can be processed independently. Rows from the first bucket are added to an in-memory hash table while the others are saved to disk. If the hash table grows beyond the memory limit (e.g., as set by [`max_bytes_in_join`](/docs/en/operations/settings/query-complexity.md/#max_bytes_in_join)), the number of buckets is increased and the assigned bucket for each row. Any rows which don’t belong to the current bucket are flushed and reassigned. Supports `INNER/LEFT/RIGHT/FULL ALL/ANY JOIN`. @@ -588,7 +588,7 @@ Possible values: ClickHouse always tries to use `partial_merge` join if possible, otherwise, it uses `hash`. *Deprecated*, same as `partial_merge,hash`. -## join_any_take_last_row {#settings-join_any_take_last_row} +## join_any_take_last_row {#join_any_take_last_row} Changes the behaviour of join operations with `ANY` strictness. @@ -607,7 +607,7 @@ See also: - [JOIN clause](../../sql-reference/statements/select/join.md/#select-join) - [Join table engine](../../engines/table-engines/special/join.md) -- [join_default_strictness](#settings-join_default_strictness) +- [join_default_strictness](#join_default_strictness) ## join_use_nulls {#join_use_nulls} @@ -879,7 +879,7 @@ Possible values: Default value: 2013265920. -## min_bytes_to_use_direct_io {#settings-min-bytes-to-use-direct-io} +## min_bytes_to_use_direct_io {#min-bytes-to-use-direct-io} The minimum data volume required for using direct I/O access to the storage disk. @@ -917,7 +917,7 @@ Possible values: Default value: `1`. -## log_queries {#settings-log-queries} +## log_queries {#log-queries} Setting up query logging. @@ -929,7 +929,7 @@ Example: log_queries=1 ``` -## log_queries_min_query_duration_ms {#settings-log-queries-min-query-duration-ms} +## log_queries_min_query_duration_ms {#log-queries-min-query-duration-ms} If enabled (non-zero), queries faster than the value of this setting will not be logged (you can think about this as a `long_query_time` for [MySQL Slow Query Log](https://dev.mysql.com/doc/refman/5.7/en/slow-query-log.html)), and this basically means that you will not find them in the following tables: @@ -944,7 +944,7 @@ Only the queries with the following type will get to the log: - Type: milliseconds - Default value: 0 (any query) -## log_queries_min_type {#settings-log-queries-min-type} +## log_queries_min_type {#log-queries-min-type} `query_log` minimal type to log. @@ -962,11 +962,11 @@ Can be used to limit which entities will go to `query_log`, say you are interest log_queries_min_type='EXCEPTION_WHILE_PROCESSING' ``` -## log_query_threads {#settings-log-query-threads} +## log_query_threads {#log-query-threads} Setting up query threads logging. -Query threads log into the [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting has effect only when [log_queries](#settings-log-queries) is true. Queries’ threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_thread_log) server configuration parameter. +Query threads log into the [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting has effect only when [log_queries](#log-queries) is true. Queries’ threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_thread_log) server configuration parameter. Possible values: @@ -981,7 +981,7 @@ Default value: `1`. log_query_threads=1 ``` -## log_query_views {#settings-log-query-views} +## log_query_views {#log-query-views} Setting up query views logging. @@ -993,7 +993,7 @@ Example: log_query_views=1 ``` -## log_formatted_queries {#settings-log-formatted-queries} +## log_formatted_queries {#log-formatted-queries} Allows to log formatted queries to the [system.query_log](../../operations/system-tables/query_log.md) system table (populates `formatted_query` column in the [system.query_log](../../operations/system-tables/query_log.md)). @@ -1004,7 +1004,7 @@ Possible values: Default value: `0`. -## log_comment {#settings-log-comment} +## log_comment {#log-comment} Specifies the value for the `log_comment` field of the [system.query_log](../system-tables/query_log.md) table and comment text for the server log. @@ -1012,7 +1012,7 @@ It can be used to improve the readability of server logs. Additionally, it helps Possible values: -- Any string no longer than [max_query_size](#settings-max_query_size). If the max_query_size is exceeded, the server throws an exception. +- Any string no longer than [max_query_size](#max_query_size). If the max_query_size is exceeded, the server throws an exception. Default value: empty string. @@ -1036,7 +1036,7 @@ Result: └─────────────┴───────────┘ ``` -## log_processors_profiles {#settings-log_processors_profiles} +## log_processors_profiles {#log_processors_profiles} Write time that processor spent during execution/waiting for data to `system.processors_profile_log` table. @@ -1045,7 +1045,7 @@ See also: - [`system.processors_profile_log`](../../operations/system-tables/processors_profile_log.md) - [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline) -## max_insert_block_size {#settings-max_insert_block_size} +## max_insert_block_size {#max_insert_block_size} The size of blocks (in a count of rows) to form for insertion into a table. This setting only applies in cases when the server forms the blocks. @@ -1079,7 +1079,7 @@ Possible values: Default value: 268435456. -## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries} +## max_replica_delay_for_distributed_queries {#max_replica_delay_for_distributed_queries} Disables lagging replicas for distributed queries. See [Replication](../../engines/table-engines/mergetree-family/replication.md). @@ -1096,7 +1096,7 @@ Default value: 300. Used when performing `SELECT` from a distributed table that points to replicated tables. -## max_threads {#settings-max_threads} +## max_threads {#max_threads} The maximum number of query processing threads, excluding threads for retrieving data from remote servers (see the ‘max_distributed_connections’ parameter). @@ -1109,7 +1109,7 @@ For queries that are completed quickly because of a LIMIT, you can set a lower The smaller the `max_threads` value, the less memory is consumed. -## max_insert_threads {#settings-max-insert-threads} +## max_insert_threads {#max-insert-threads} The maximum number of threads to execute the `INSERT SELECT` query. @@ -1120,7 +1120,7 @@ Possible values: Default value: 0. -Parallel `INSERT SELECT` has effect only if the `SELECT` part is executed in parallel, see [max_threads](#settings-max_threads) setting. +Parallel `INSERT SELECT` has effect only if the `SELECT` part is executed in parallel, see [max_threads](#max_threads) setting. Higher values will lead to higher memory usage. ## max_compress_block_size {#max-compress-block-size} @@ -1149,7 +1149,7 @@ We are writing a URL column with the String type (average size of 60 bytes per v This is an expert-level setting, and you shouldn't change it if you're just getting started with ClickHouse. ::: -## max_query_size {#settings-max_query_size} +## max_query_size {#max_query_size} The maximum number of bytes of a query string parsed by the SQL parser. Data in the VALUES clause of INSERT queries is processed by a separate stream parser (that consumes O(1) RAM) and not affected by this restriction. @@ -1393,7 +1393,7 @@ Default value: 5000. ## stream_flush_interval_ms {#stream-flush-interval-ms} -Works for tables with streaming in the case of a timeout, or when a thread generates [max_insert_block_size](#settings-max_insert_block_size) rows. +Works for tables with streaming in the case of a timeout, or when a thread generates [max_insert_block_size](#max_insert_block_size) rows. The default value is 7500. @@ -1405,7 +1405,7 @@ Timeout for polling data from/to streaming storages. Default value: 500. -## load_balancing {#settings-load_balancing} +## load_balancing {#load_balancing} Specifies the algorithm of replicas selection that is used for distributed query processing. @@ -1419,7 +1419,7 @@ ClickHouse supports the following algorithms of choosing replicas: See also: -- [distributed_replica_max_ignored_errors](#settings-distributed_replica_max_ignored_errors) +- [distributed_replica_max_ignored_errors](#distributed_replica_max_ignored_errors) ### Random (by Default) {#load_balancing-random} @@ -1473,20 +1473,20 @@ load_balancing = round_robin This algorithm uses a round-robin policy across replicas with the same number of errors (only the queries with `round_robin` policy is accounted). -## prefer_localhost_replica {#settings-prefer-localhost-replica} +## prefer_localhost_replica {#prefer-localhost-replica} Enables/disables preferable using the localhost replica when processing distributed queries. Possible values: - 1 — ClickHouse always sends a query to the localhost replica if it exists. -- 0 — ClickHouse uses the balancing strategy specified by the [load_balancing](#settings-load_balancing) setting. +- 0 — ClickHouse uses the balancing strategy specified by the [load_balancing](#load_balancing) setting. Default value: 1. :::note -Disable this setting if you use [max_parallel_replicas](#settings-max_parallel_replicas) without [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key). -If [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key) is set, disable this setting only if it's used on a cluster with multiple shards containing multiple replicas. +Disable this setting if you use [max_parallel_replicas](#max_parallel_replicas) without [parallel_replicas_custom_key](#parallel_replicas_custom_key). +If [parallel_replicas_custom_key](#parallel_replicas_custom_key) is set, disable this setting only if it's used on a cluster with multiple shards containing multiple replicas. If it's used on a cluster with a single shard and multiple replicas, disabling this setting will have negative effects. ::: @@ -1500,7 +1500,7 @@ See the section “WITH TOTALS modifier”. The threshold for `totals_mode = 'auto'`. See the section “WITH TOTALS modifier”. -## max_parallel_replicas {#settings-max_parallel_replicas} +## max_parallel_replicas {#max_parallel_replicas} The maximum number of replicas for each shard when executing a query. @@ -1527,23 +1527,23 @@ A query may be processed faster if it is executed on several servers in parallel - The sampling key is an expression that is expensive to calculate. - The cluster latency distribution has a long tail, so that querying more servers increases the query overall latency. -### Parallel processing using [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key) +### Parallel processing using [parallel_replicas_custom_key](#parallel_replicas_custom_key) This setting is useful for any replicated table. -## parallel_replicas_custom_key {#settings-parallel_replicas_custom_key} +## parallel_replicas_custom_key {#parallel_replicas_custom_key} An arbitrary integer expression that can be used to split work between replicas for a specific table. The value can be any integer expression. -A query may be processed faster if it is executed on several servers in parallel but it depends on the used [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key) -and [parallel_replicas_custom_key_filter_type](#settings-parallel_replicas_custom_key_filter_type). +A query may be processed faster if it is executed on several servers in parallel but it depends on the used [parallel_replicas_custom_key](#parallel_replicas_custom_key) +and [parallel_replicas_custom_key_filter_type](#parallel_replicas_custom_key_filter_type). Simple expressions using primary keys are preferred. If the setting is used on a cluster that consists of a single shard with multiple replicas, those replicas will be converted into virtual shards. Otherwise, it will behave same as for `SAMPLE` key, it will use multiple replicas of each shard. -## parallel_replicas_custom_key_filter_type {#settings-parallel_replicas_custom_key_filter_type} +## parallel_replicas_custom_key_filter_type {#parallel_replicas_custom_key_filter_type} How to use `parallel_replicas_custom_key` expression for splitting work between replicas. @@ -1732,7 +1732,7 @@ Possible values: Default value: 0 (no restriction). -## insert_quorum {#settings-insert_quorum} +## insert_quorum {#insert_quorum} Enables the quorum writes. @@ -1746,7 +1746,7 @@ Quorum writes `INSERT` succeeds only when ClickHouse manages to correctly write data to the `insert_quorum` of replicas during the `insert_quorum_timeout`. If for any reason the number of replicas with successful writes does not reach the `insert_quorum`, the write is considered failed and ClickHouse will delete the inserted block from all the replicas where data has already been written. -When `insert_quorum_parallel` is disabled, all replicas in the quorum are consistent, i.e. they contain data from all previous `INSERT` queries (the `INSERT` sequence is linearized). When reading data written using `insert_quorum` and `insert_quorum_parallel` is disabled, you can turn on sequential consistency for `SELECT` queries using [select_sequential_consistency](#settings-select_sequential_consistency). +When `insert_quorum_parallel` is disabled, all replicas in the quorum are consistent, i.e. they contain data from all previous `INSERT` queries (the `INSERT` sequence is linearized). When reading data written using `insert_quorum` and `insert_quorum_parallel` is disabled, you can turn on sequential consistency for `SELECT` queries using [select_sequential_consistency](#select_sequential_consistency). ClickHouse generates an exception: @@ -1755,11 +1755,11 @@ ClickHouse generates an exception: See also: -- [insert_quorum_timeout](#settings-insert_quorum_timeout) -- [insert_quorum_parallel](#settings-insert_quorum_parallel) -- [select_sequential_consistency](#settings-select_sequential_consistency) +- [insert_quorum_timeout](#insert_quorum_timeout) +- [insert_quorum_parallel](#insert_quorum_parallel) +- [select_sequential_consistency](#select_sequential_consistency) -## insert_quorum_timeout {#settings-insert_quorum_timeout} +## insert_quorum_timeout {#insert_quorum_timeout} Write to a quorum timeout in milliseconds. If the timeout has passed and no write has taken place yet, ClickHouse will generate an exception and the client must repeat the query to write the same block to the same or any other replica. @@ -1767,11 +1767,11 @@ Default value: 600 000 milliseconds (ten minutes). See also: -- [insert_quorum](#settings-insert_quorum) -- [insert_quorum_parallel](#settings-insert_quorum_parallel) -- [select_sequential_consistency](#settings-select_sequential_consistency) +- [insert_quorum](#insert_quorum) +- [insert_quorum_parallel](#insert_quorum_parallel) +- [select_sequential_consistency](#select_sequential_consistency) -## insert_quorum_parallel {#settings-insert_quorum_parallel} +## insert_quorum_parallel {#insert_quorum_parallel} Enables or disables parallelism for quorum `INSERT` queries. If enabled, additional `INSERT` queries can be sent while previous queries have not yet finished. If disabled, additional writes to the same table will be rejected. @@ -1784,11 +1784,11 @@ Default value: 1. See also: -- [insert_quorum](#settings-insert_quorum) -- [insert_quorum_timeout](#settings-insert_quorum_timeout) -- [select_sequential_consistency](#settings-select_sequential_consistency) +- [insert_quorum](#insert_quorum) +- [insert_quorum_timeout](#insert_quorum_timeout) +- [select_sequential_consistency](#select_sequential_consistency) -## select_sequential_consistency {#settings-select_sequential_consistency} +## select_sequential_consistency {#select_sequential_consistency} Enables or disables sequential consistency for `SELECT` queries. Requires `insert_quorum_parallel` to be disabled (enabled by default). @@ -1807,11 +1807,11 @@ When `insert_quorum_parallel` is enabled (the default), then `select_sequential_ See also: -- [insert_quorum](#settings-insert_quorum) -- [insert_quorum_timeout](#settings-insert_quorum_timeout) -- [insert_quorum_parallel](#settings-insert_quorum_parallel) +- [insert_quorum](#insert_quorum) +- [insert_quorum_timeout](#insert_quorum_timeout) +- [insert_quorum_parallel](#insert_quorum_parallel) -## insert_deduplicate {#settings-insert-deduplicate} +## insert_deduplicate {#insert-deduplicate} Enables or disables block deduplication of `INSERT` (for Replicated\* tables). @@ -1938,7 +1938,7 @@ For the replicated tables, by default, only 10000 of the most recent inserts for We recommend enabling the [async_block_ids_cache](merge-tree-settings.md/#use-async-block-ids-cache) to increase the efficiency of deduplication. This function does not work for non-replicated tables. -## deduplicate_blocks_in_dependent_materialized_views {#settings-deduplicate-blocks-in-dependent-materialized-views} +## deduplicate_blocks_in_dependent_materialized_views {#deduplicate-blocks-in-dependent-materialized-views} Enables or disables the deduplication check for materialized views that receive data from Replicated\* tables. @@ -2048,7 +2048,7 @@ Possible values: Default value: 10000 -## max_network_bytes {#settings-max-network-bytes} +## max_network_bytes {#max-network-bytes} Limits the data volume (in bytes) that is received or transmitted over the network when executing a query. This setting applies to every individual query. @@ -2059,7 +2059,7 @@ Possible values: Default value: 0. -## max_network_bandwidth {#settings-max-network-bandwidth} +## max_network_bandwidth {#max-network-bandwidth} Limits the speed of the data exchange over the network in bytes per second. This setting applies to every query. @@ -2070,7 +2070,7 @@ Possible values: Default value: 0. -## max_network_bandwidth_for_user {#settings-max-network-bandwidth-for-user} +## max_network_bandwidth_for_user {#max-network-bandwidth-for-user} Limits the speed of the data exchange over the network in bytes per second. This setting applies to all concurrently running queries performed by a single user. @@ -2081,7 +2081,7 @@ Possible values: Default value: 0. -## max_network_bandwidth_for_all_users {#settings-max-network-bandwidth-for-all-users} +## max_network_bandwidth_for_all_users {#max-network-bandwidth-for-all-users} Limits the speed that data is exchanged at over the network in bytes per second. This setting applies to all concurrently running queries on the server. @@ -2092,7 +2092,7 @@ Possible values: Default value: 0. -## count_distinct_implementation {#settings-count_distinct_implementation} +## count_distinct_implementation {#count_distinct_implementation} Specifies which of the `uniq*` functions should be used to perform the [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference/count.md/#agg_function-count) construction. @@ -2106,7 +2106,7 @@ Possible values: Default value: `uniqExact`. -## skip_unavailable_shards {#settings-skip_unavailable_shards} +## skip_unavailable_shards {#skip_unavailable_shards} Enables or disables silently skipping of unavailable shards. @@ -2270,7 +2270,7 @@ Possible values: Default value: 0 -## force_optimize_skip_unused_shards_nesting {#settings-force_optimize_skip_unused_shards_nesting} +## force_optimize_skip_unused_shards_nesting {#force_optimize_skip_unused_shards_nesting} Controls [`force_optimize_skip_unused_shards`](#force-optimize-skip-unused-shards) (hence still requires [`force_optimize_skip_unused_shards`](#force-optimize-skip-unused-shards)) depends on the nesting level of the distributed query (case when you have `Distributed` table that look into another `Distributed` table). @@ -2400,7 +2400,7 @@ Enables caching of rows number during count from files in table functions `file` Enabled by default. -## distributed_replica_error_half_life {#settings-distributed_replica_error_half_life} +## distributed_replica_error_half_life {#distributed_replica_error_half_life} - Type: seconds - Default value: 60 seconds @@ -2411,10 +2411,10 @@ See also: - [load_balancing](#load_balancing-round_robin) - [Table engine Distributed](../../engines/table-engines/special/distributed.md) -- [distributed_replica_error_cap](#settings-distributed_replica_error_cap) -- [distributed_replica_max_ignored_errors](#settings-distributed_replica_max_ignored_errors) +- [distributed_replica_error_cap](#distributed_replica_error_cap) +- [distributed_replica_max_ignored_errors](#distributed_replica_max_ignored_errors) -## distributed_replica_error_cap {#settings-distributed_replica_error_cap} +## distributed_replica_error_cap {#distributed_replica_error_cap} - Type: unsigned int - Default value: 1000 @@ -2425,10 +2425,10 @@ See also: - [load_balancing](#load_balancing-round_robin) - [Table engine Distributed](../../engines/table-engines/special/distributed.md) -- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life) -- [distributed_replica_max_ignored_errors](#settings-distributed_replica_max_ignored_errors) +- [distributed_replica_error_half_life](#distributed_replica_error_half_life) +- [distributed_replica_max_ignored_errors](#distributed_replica_max_ignored_errors) -## distributed_replica_max_ignored_errors {#settings-distributed_replica_max_ignored_errors} +## distributed_replica_max_ignored_errors {#distributed_replica_max_ignored_errors} - Type: unsigned int - Default value: 0 @@ -2439,7 +2439,7 @@ See also: - [load_balancing](#load_balancing-round_robin) - [Table engine Distributed](../../engines/table-engines/special/distributed.md) -- [distributed_replica_error_cap](#settings-distributed_replica_error_cap) +- [distributed_replica_error_cap](#distributed_replica_error_cap) - [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life) ## distributed_directory_monitor_sleep_time_ms {#distributed_directory_monitor_sleep_time_ms} @@ -2595,7 +2595,7 @@ Possible values: Default value: 0. -## allow_introspection_functions {#settings-allow_introspection_functions} +## allow_introspection_functions {#allow_introspection_functions} Enables or disables [introspection functions](../../sql-reference/functions/introspection.md) for query profiling. @@ -3136,7 +3136,7 @@ Do not enable this feature in version `<= 21.8`. It's not properly implemented a ## aggregate_functions_null_for_empty {#aggregate_functions_null_for_empty} Enables or disables rewriting all aggregate functions in a query, adding [-OrNull](../../sql-reference/aggregate-functions/combinators.md/#agg-functions-combinator-ornull) suffix to them. Enable it for SQL standard compatibility. -It is implemented via query rewrite (similar to [count_distinct_implementation](#settings-count_distinct_implementation) setting) to get consistent results for distributed queries. +It is implemented via query rewrite (similar to [count_distinct_implementation](#count_distinct_implementation) setting) to get consistent results for distributed queries. Possible values: @@ -4609,7 +4609,7 @@ Default: 0 ## rewrite_count_distinct_if_with_count_distinct_implementation -Allows you to rewrite `countDistcintIf` with [count_distinct_implementation](#settings-count_distinct_implementation) setting. +Allows you to rewrite `countDistcintIf` with [count_distinct_implementation](#count_distinct_implementation) setting. Possible values: diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 80f728a33df..f55b6114e07 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -123,7 +123,7 @@ LAYOUT(...) -- Memory layout configuration LIFETIME(...) -- Lifetime of dictionary in memory ``` -## Storing Dictionaries in Memory {#storig-dictionaries-in-memory} +## Storing Dictionaries in Memory {#storing-dictionaries-in-memory} There are a variety of ways to store dictionaries in memory. diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index dae8ff6d396..59af1b909ca 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -657,7 +657,7 @@ SELECT arraySlice([1, 2, NULL, 4, 5], 2, 3) AS res; Array elements set to `NULL` are handled as normal values. -## arraySort(\[func,\] arr, …) {#array_functions-sort} +## arraySort(\[func,\] arr, …) {#sort} Sorts the elements of the `arr` array in ascending order. If the `func` function is specified, sorting order is determined by the result of the `func` function applied to the elements of the array. If `func` accepts multiple arguments, the `arraySort` function is passed several arrays that the arguments of `func` will correspond to. Detailed examples are shown at the end of `arraySort` description. @@ -716,7 +716,7 @@ SELECT arraySort((x) -> -x, [1, 2, 3]) as res; └─────────┘ ``` -For each element of the source array, the lambda function returns the sorting key, that is, \[1 –\> -1, 2 –\> -2, 3 –\> -3\]. Since the `arraySort` function sorts the keys in ascending order, the result is \[3, 2, 1\]. Thus, the `(x) –> -x` lambda function sets the [descending order](#array_functions-reverse-sort) in a sorting. +For each element of the source array, the lambda function returns the sorting key, that is, \[1 –\> -1, 2 –\> -2, 3 –\> -3\]. Since the `arraySort` function sorts the keys in ascending order, the result is \[3, 2, 1\]. Thus, the `(x) –> -x` lambda function sets the [descending order](#reverse-sort) in a sorting. The lambda function can accept multiple arguments. In this case, you need to pass the `arraySort` function several arrays of identical length that the arguments of lambda function will correspond to. The resulting array will consist of elements from the first input array; elements from the next input array(s) specify the sorting keys. For example: @@ -762,7 +762,7 @@ To improve sorting efficiency, the [Schwartzian transform](https://en.wikipedia. Same as `arraySort` with additional `limit` argument allowing partial sorting. Returns an array of the same size as the original array where elements in range `[1..limit]` are sorted in ascending order. Remaining elements `(limit..N]` shall contain elements in unspecified order. -## arrayReverseSort(\[func,\] arr, …) {#array_functions-reverse-sort} +## arrayReverseSort(\[func,\] arr, …) {#reverse-sort} Sorts the elements of the `arr` array in descending order. If the `func` function is specified, `arr` is sorted according to the result of the `func` function applied to the elements of the array, and then the sorted array is reversed. If `func` accepts multiple arguments, the `arrayReverseSort` function is passed several arrays that the arguments of `func` will correspond to. Detailed examples are shown at the end of `arrayReverseSort` description. diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index d3b39dd4147..a3883cd22ce 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -239,7 +239,7 @@ int32samoa: 1546300800 **See Also** -- [formatDateTime](#date_time_functions-formatDateTime) - supports non-constant timezone. +- [formatDateTime](#formatDateTime) - supports non-constant timezone. - [toString](type-conversion-functions.md#tostring) - supports non-constant timezone. ## timeZoneOf @@ -1628,7 +1628,7 @@ SELECT timeSlots(toDateTime64('1980-12-12 21:01:02.1234', 4, 'UTC'), toDecimal64 └───────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` -## formatDateTime {#date_time_functions-formatDateTime} +## formatDateTime {#formatDateTime} Formats a Time according to the given Format string. Format is a constant expression, so you cannot have multiple formats for a single result column. @@ -1753,7 +1753,7 @@ LIMIT 10 - [formatDateTimeInJodaSyntax](##formatDateTimeInJodaSyntax) -## formatDateTimeInJodaSyntax {#date_time_functions-formatDateTimeInJodaSyntax} +## formatDateTimeInJodaSyntax {#formatDateTimeInJodaSyntax} Similar to formatDateTime, except that it formats datetime in Joda style instead of MySQL style. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 556fe622c27..7276437ec82 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -19,7 +19,7 @@ halfMD5(par1, ...) ``` The function is relatively slow (5 million short strings per second per processor core). -Consider using the [sipHash64](#hash_functions-siphash64) function instead. +Consider using the [sipHash64](#siphash64) function instead. **Arguments** @@ -45,13 +45,13 @@ SELECT halfMD5(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00') Calculates the MD4 from a string and returns the resulting set of bytes as FixedString(16). -## MD5 {#hash_functions-md5} +## MD5 {#md5} Calculates the MD5 from a string and returns the resulting set of bytes as FixedString(16). If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the ‘sipHash128’ function instead. If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))). -## sipHash64 {#hash_functions-siphash64} +## sipHash64 {#siphash64} Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value. @@ -59,7 +59,7 @@ Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value. sipHash64(par1,...) ``` -This is a cryptographic hash function. It works at least three times faster than the [MD5](#hash_functions-md5) hash function. +This is a cryptographic hash function. It works at least three times faster than the [MD5](#md5) hash function. The function [interprets](/docs/en/sql-reference/functions/type-conversion-functions.md/#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. It then combines the hashes by the following algorithm: @@ -91,7 +91,7 @@ SELECT sipHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00 ## sipHash64Keyed -Same as [sipHash64](#hash_functions-siphash64) but additionally takes an explicit key argument instead of using a fixed key. +Same as [sipHash64](#siphash64) but additionally takes an explicit key argument instead of using a fixed key. **Syntax** @@ -101,7 +101,7 @@ sipHash64Keyed((k0, k1), par1,...) **Arguments** -Same as [sipHash64](#hash_functions-siphash64), but the first argument is a tuple of two UInt64 values representing the key. +Same as [sipHash64](#siphash64), but the first argument is a tuple of two UInt64 values representing the key. **Returned value** @@ -123,12 +123,12 @@ SELECT sipHash64Keyed((506097522914230528, 1084818905618843912), array('e','x',' ## sipHash128 -Like [sipHash64](#hash_functions-siphash64) but produces a 128-bit hash value, i.e. the final xor-folding state is done up to 128 bits. +Like [sipHash64](#siphash64) but produces a 128-bit hash value, i.e. the final xor-folding state is done up to 128 bits. :::note This 128-bit variant differs from the reference implementation and it's weaker. This version exists because, when it was written, there was no official 128-bit extension for SipHash. -New projects should probably use [sipHash128Reference](#hash_functions-siphash128reference). +New projects should probably use [sipHash128Reference](#siphash128reference). ::: **Syntax** @@ -139,7 +139,7 @@ sipHash128(par1,...) **Arguments** -Same as for [sipHash64](#hash_functions-siphash64). +Same as for [sipHash64](#siphash64). **Returned value** @@ -163,12 +163,12 @@ Result: ## sipHash128Keyed -Same as [sipHash128](#hash_functions-siphash128) but additionally takes an explicit key argument instead of using a fixed key. +Same as [sipHash128](#siphash128) but additionally takes an explicit key argument instead of using a fixed key. :::note This 128-bit variant differs from the reference implementation and it's weaker. This version exists because, when it was written, there was no official 128-bit extension for SipHash. -New projects should probably use [sipHash128ReferenceKeyed](#hash_functions-siphash128referencekeyed). +New projects should probably use [sipHash128ReferenceKeyed](#siphash128referencekeyed). ::: **Syntax** @@ -179,7 +179,7 @@ sipHash128Keyed((k0, k1), par1,...) **Arguments** -Same as [sipHash128](#hash_functions-siphash128), but the first argument is a tuple of two UInt64 values representing the key. +Same as [sipHash128](#siphash128), but the first argument is a tuple of two UInt64 values representing the key. **Returned value** @@ -203,7 +203,7 @@ Result: ## sipHash128Reference -Like [sipHash128](#hash_functions-siphash128) but implements the 128-bit algorithm from the original authors of SipHash. +Like [sipHash128](#siphash128) but implements the 128-bit algorithm from the original authors of SipHash. **Syntax** @@ -213,7 +213,7 @@ sipHash128Reference(par1,...) **Arguments** -Same as for [sipHash128](#hash_functions-siphash128). +Same as for [sipHash128](#siphash128). **Returned value** @@ -237,7 +237,7 @@ Result: ## sipHash128ReferenceKeyed -Same as [sipHash128Reference](#hash_functions-siphash128reference) but additionally takes an explicit key argument instead of using a fixed key. +Same as [sipHash128Reference](#siphash128reference) but additionally takes an explicit key argument instead of using a fixed key. **Syntax** @@ -247,7 +247,7 @@ sipHash128ReferenceKeyed((k0, k1), par1,...) **Arguments** -Same as [sipHash128Reference](#hash_functions-siphash128reference), but the first argument is a tuple of two UInt64 values representing the key. +Same as [sipHash128Reference](#siphash128reference), but the first argument is a tuple of two UInt64 values representing the key. **Returned value** @@ -536,7 +536,7 @@ Calculates `HiveHash` from a string. SELECT hiveHash('') ``` -This is just [JavaHash](#hash_functions-javahash) with zeroed out sign bit. This function is used in [Apache Hive](https://en.wikipedia.org/wiki/Apache_Hive) for versions before 3.0. This hash function is neither fast nor having a good quality. The only reason to use it is when this algorithm is already used in another system and you have to calculate exactly the same result. +This is just [JavaHash](#javahash) with zeroed out sign bit. This function is used in [Apache Hive](https://en.wikipedia.org/wiki/Apache_Hive) for versions before 3.0. This hash function is neither fast nor having a good quality. The only reason to use it is when this algorithm is already used in another system and you have to calculate exactly the same result. **Returned value** From baf1016e4b556400c624b3a497863a9dd96af586 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Sep 2023 18:39:21 +0000 Subject: [PATCH 55/93] More --- .../table-engines/integrations/s3queue.md | 16 ++++++++-------- .../settings/permissions-for-queries.md | 2 +- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 8988d8cde62..50330962b86 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -63,7 +63,7 @@ SETTINGS mode = 'ordered'; ``` -## Settings {#s3queue-settings} +## Settings {#settings} ### mode {#mode} @@ -93,7 +93,7 @@ Possible values: Default value: `/`. -### s3queue_loading_retries {#s3queue_loading_retries} +### s3queue_loading_retries {#loading_retries} Retry file loading up to specified number of times. By default, there are no retries. Possible values: @@ -102,7 +102,7 @@ Possible values: Default value: `0`. -### s3queue_polling_min_timeout_ms {#s3queue_polling_min_timeout_ms} +### s3queue_polling_min_timeout_ms {#polling_min_timeout_ms} Minimal timeout before next polling (in milliseconds). @@ -112,7 +112,7 @@ Possible values: Default value: `1000`. -### s3queue_polling_max_timeout_ms {#s3queue_polling_max_timeout_ms} +### s3queue_polling_max_timeout_ms {#polling_max_timeout_ms} Maximum timeout before next polling (in milliseconds). @@ -122,7 +122,7 @@ Possible values: Default value: `10000`. -### s3queue_polling_backoff_ms {#s3queue_polling_backoff_ms} +### s3queue_polling_backoff_ms {#polling_backoff_ms} Polling backoff (in milliseconds). @@ -132,7 +132,7 @@ Possible values: Default value: `0`. -### s3queue_tracked_files_limit {#s3queue_tracked_files_limit} +### s3queue_tracked_files_limit {#tracked_files_limit} Allows to limit the number of Zookeeper nodes if the 'unordered' mode is used, does nothing for 'ordered' mode. If limit reached the oldest processed files will be deleted from ZooKeeper node and processed again. @@ -143,7 +143,7 @@ Possible values: Default value: `1000`. -### s3queue_tracked_file_ttl_sec {#s3queue_tracked_file_ttl_sec} +### s3queue_tracked_file_ttl_sec {#tracked_file_ttl_sec} Maximum number of seconds to store processed files in ZooKeeper node (store forever by default) for 'unordered' mode, does nothing for 'ordered' mode. After the specified number of seconds, the file will be re-imported. @@ -154,7 +154,7 @@ Possible values: Default value: `0`. -### s3queue_polling_size {#s3queue_polling_size} +### s3queue_polling_size {#polling_size} Maximum files to fetch from S3 with SELECT or in background task. Engine takes files for processing from S3 in batches. diff --git a/docs/en/operations/settings/permissions-for-queries.md b/docs/en/operations/settings/permissions-for-queries.md index 9e9c564d426..bdcd1139418 100644 --- a/docs/en/operations/settings/permissions-for-queries.md +++ b/docs/en/operations/settings/permissions-for-queries.md @@ -48,7 +48,7 @@ Setting `readonly = 1` prohibits the user from changing settings. There is a way ::: -## allow_ddl {#settings_allow_ddl} +## allow_ddl {#allow_ddl} Allows or denies [DDL](https://en.wikipedia.org/wiki/Data_definition_language) queries. From de9f4be7bdc5b00c2727dce5b5c6e84e7c732cba Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 19 Sep 2023 00:04:25 +0200 Subject: [PATCH 56/93] Update system.md --- docs/en/sql-reference/statements/system.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 8d7cdb80c2b..4e1e7aa6a49 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -347,6 +347,8 @@ Waits until the specified [replicated database](https://clickhouse.com/docs/en/e **Syntax** ```sql SYSTEM SYNC DATABASE REPLICA replicated_database_name; +``` + ### RESTART REPLICA Provides possibility to reinitialize Zookeeper session's state for `ReplicatedMergeTree` table, will compare current state with Zookeeper as source of truth and add tasks to Zookeeper queue if needed. From 142a4631f14f02ad198e9c67700f44ec1e001dae Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 19 Sep 2023 11:22:10 +0200 Subject: [PATCH 57/93] Updated to clear current_row_sources when cleanedup rows are added --- .../Merges/Algorithms/ReplacingSortedAlgorithm.cpp | 7 ++++++- .../00577_replacing_merge_tree_vertical_merge.sql | 5 +++-- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp index 18f144bee3d..0c0598171b3 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp @@ -28,7 +28,6 @@ ReplacingSortedAlgorithm::ReplacingSortedAlgorithm( , cleanup(cleanup_) , cleanedup_rows_count(cleanedup_rows_count_) { - if (!is_deleted_column.empty()) is_deleted_column_number = header_.getPositionByName(is_deleted_column); if (!version_column.empty()) @@ -84,7 +83,10 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() if (!cleanup || !value) insertRow(); else if (cleanup && cleanedup_rows_count != nullptr) + { *cleanedup_rows_count += current_row_sources.size(); + current_row_sources.resize(0); + } } else insertRow(); @@ -142,7 +144,10 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() if (!cleanup || !value) insertRow(); else if (cleanup && cleanedup_rows_count != nullptr) + { *cleanedup_rows_count += current_row_sources.size(); + current_row_sources.resize(0); + } } else insertRow(); diff --git a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql index 931297fdd3b..e3c1bb10426 100644 --- a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql +++ b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql @@ -7,7 +7,7 @@ create table tab_00577 (date Date, version UInt64, val UInt64) engine = Replacin insert into tab_00577 values ('2018-01-01', 2, 2), ('2018-01-01', 1, 1); insert into tab_00577 values ('2018-01-01', 0, 0); select * from tab_00577 order by version; -OPTIMIZE TABLE tab_00577; +OPTIMIZE TABLE tab_00577 FINAL CLEANUP; select * from tab_00577; drop table tab_00577; @@ -15,7 +15,8 @@ drop table tab_00577; DROP TABLE IF EXISTS testCleanupR1; CREATE TABLE testCleanupR1 (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{database}/tables/test_cleanup/', 'r1', version, is_deleted) - ORDER BY uid SETTINGS enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 0; + ORDER BY uid SETTINGS enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 0, vertical_merge_algorithm_min_columns_to_activate = 0, min_rows_for_wide_part = 0, + min_bytes_for_wide_part = 0; INSERT INTO testCleanupR1 (*) VALUES ('d1', 1, 0),('d2', 1, 0),('d3', 1, 0),('d4', 1, 0); INSERT INTO testCleanupR1 (*) VALUES ('d3', 2, 1); INSERT INTO testCleanupR1 (*) VALUES ('d1', 2, 1); From 834ec7b3823a8a36ea283e30adfacc82026cf3ed Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 14 Sep 2023 09:26:30 +0800 Subject: [PATCH 58/93] reuse GeneratorJSONPath --- src/Functions/FunctionSQLJSON.h | 14 +++++++------- .../JSONPath/Generator/GeneratorJSONPath.h | 10 ++++++++++ 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index 55ffd817bb1..1786b613f98 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -199,6 +199,7 @@ public: /// Parse JSON for every row Impl impl; + GeneratorJSONPath generator_json_path(res); for (const auto i : collections::range(0, input_rows_count)) { std::string_view json{ @@ -208,7 +209,9 @@ public: bool added_to_column = false; if (document_ok) { - added_to_column = impl.insertResultToColumn(*to, document, res, context); + // Instead of creating a new generator for each row, we can reuse the same one. + generator_json_path.reinitialize(); + added_to_column = impl.insertResultToColumn(*to, document, generator_json_path, context); } if (!added_to_column) { @@ -287,9 +290,8 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const ContextPtr &) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const ContextPtr &) { - GeneratorJSONPath generator_json_path(query_ptr); Element current_element = root; VisitorStatus status; while ((status = generator_json_path.getNextItem(current_element)) != VisitorStatus::Exhausted) @@ -337,9 +339,8 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const ContextPtr & context) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const ContextPtr & context) { - GeneratorJSONPath generator_json_path(query_ptr); Element current_element = root; VisitorStatus status; @@ -405,11 +406,10 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const ContextPtr &) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const ContextPtr &) { ColumnString & col_str = assert_cast(dest); - GeneratorJSONPath generator_json_path(query_ptr); Element current_element = root; VisitorStatus status; bool success = false; diff --git a/src/Functions/JSONPath/Generator/GeneratorJSONPath.h b/src/Functions/JSONPath/Generator/GeneratorJSONPath.h index be02656b07d..de62be98d31 100644 --- a/src/Functions/JSONPath/Generator/GeneratorJSONPath.h +++ b/src/Functions/JSONPath/Generator/GeneratorJSONPath.h @@ -105,6 +105,16 @@ public: } } + void reinitialize() + { + while (current_visitor >= 0) + { + visitors[current_visitor]->reinitialize(); + current_visitor--; + } + current_visitor = 0; + } + private: bool updateVisitorsForNextRun() { From c7ddbab9bc49757e41b737e85cbea8e1176a47bb Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 19 Sep 2023 13:06:19 +0300 Subject: [PATCH 59/93] Updated implementation --- src/DataTypes/Utils.cpp | 231 ++++++++++++++++++++++++ src/DataTypes/Utils.h | 19 ++ src/Functions/FunctionHelpers.cpp | 1 - src/Storages/MergeTree/KeyCondition.cpp | 34 +++- 4 files changed, 275 insertions(+), 10 deletions(-) create mode 100644 src/DataTypes/Utils.cpp create mode 100644 src/DataTypes/Utils.h diff --git a/src/DataTypes/Utils.cpp b/src/DataTypes/Utils.cpp new file mode 100644 index 00000000000..0168c6d256c --- /dev/null +++ b/src/DataTypes/Utils.cpp @@ -0,0 +1,231 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_type) +{ + auto from_which_type = WhichDataType(from_type->getTypeId()); + bool to_type_was_nullable = isNullableOrLowCardinalityNullable(to_type); + auto to_type_unwrapped = removeNullable(removeLowCardinality(to_type)); + + if (from_type->equals(*to_type)) + return true; + + auto to_which_type = WhichDataType(to_type->getTypeId()); + + switch (from_which_type.idx) + { + case TypeIndex::UInt8: + case TypeIndex::UInt16: + case TypeIndex::UInt32: + case TypeIndex::UInt64: + case TypeIndex::UInt128: + case TypeIndex::UInt256: + { + if (to_which_type.isUInt() && + to_type_unwrapped->getSizeOfValueInMemory() >= from_type->getSizeOfValueInMemory()) + return true; + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Int8: + case TypeIndex::Int16: + case TypeIndex::Int32: + case TypeIndex::Int64: + case TypeIndex::Int128: + case TypeIndex::Int256: + { + if (to_which_type.isInt() && + to_type_unwrapped->getSizeOfValueInMemory() >= from_type->getSizeOfValueInMemory()) + return true; + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Float32: + { + if (to_which_type.isFloat64() || to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Float64: + case TypeIndex::Date: + case TypeIndex::Date32: + case TypeIndex::DateTime: + case TypeIndex::DateTime64: + case TypeIndex::FixedString: + case TypeIndex::Enum8: + case TypeIndex::Enum16: + case TypeIndex::IPv6: + { + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Decimal32: + case TypeIndex::Decimal64: + case TypeIndex::Decimal128: + case TypeIndex::Decimal256: + { + if (to_which_type.isDecimal()) + { + auto from_type_decimal_precision = getDecimalPrecision(*from_type); + auto to_type_decimal_precision = getDecimalPrecision(*to_type_unwrapped); + if (from_type_decimal_precision > to_type_decimal_precision) + return false; + + auto from_type_decimal_scale = getDecimalScale(*from_type); + auto to_type_decimal_scale = getDecimalScale(*to_type_unwrapped); + if (from_type_decimal_scale > to_type_decimal_scale) + return false; + + return true; + } + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::UUID: + { + if (to_which_type.isUInt128() || to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::IPv4: + { + if (to_which_type.isUInt32() || to_which_type.isUInt64() || to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Nullable: + { + if (to_type_was_nullable) + { + const auto & from_type_nullable = assert_cast(*from_type); + return canBeSafelyCasted(from_type_nullable.getNestedType(), to_type_unwrapped); + } + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::LowCardinality: + { + const auto & from_type_low_cardinality = assert_cast(*from_type); + return canBeSafelyCasted(from_type_low_cardinality.getDictionaryType(), to_type_unwrapped); + } + case TypeIndex::Array: + { + if (to_which_type.isArray()) + { + const auto & from_type_array = assert_cast(*from_type); + const auto & to_type_array = assert_cast(*to_type_unwrapped); + return canBeSafelyCasted(from_type_array.getNestedType(), to_type_array.getNestedType()); + } + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Map: + { + if (to_which_type.isMap()) + { + const auto & from_type_map = assert_cast(*from_type); + const auto & to_type_map = assert_cast(*to_type_unwrapped); + if (!canBeSafelyCasted(from_type_map.getKeyType(), to_type_map.getKeyType())) + return false; + + if (!canBeSafelyCasted(from_type_map.getValueType(), to_type_map.getValueType())) + return false; + + return true; + } + + if (to_which_type.isArray()) + { + // Map nested type is Array(Tuple(key_type, value_type)) + const auto & from_type_map = assert_cast(*from_type); + const auto & to_type_array = assert_cast(*to_type_unwrapped); + const auto * to_type_nested_tuple_type = typeid_cast(to_type_array.getNestedType().get()); + if (!to_type_nested_tuple_type) + return false; + + const auto & to_type_tuple_elements = to_type_nested_tuple_type->getElements(); + if (to_type_tuple_elements.size() != 2) + return false; + + if (!canBeSafelyCasted(from_type_map.getKeyType(), to_type_tuple_elements[0])) + return false; + + if (!canBeSafelyCasted(from_type_map.getValueType(), to_type_tuple_elements[1])) + return false; + + return true; + } + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Tuple: + { + if (to_which_type.isTuple()) + { + const auto & from_type_tuple = assert_cast(*from_type); + const auto & to_type_tuple = assert_cast(*to_type_unwrapped); + + const auto & from_tuple_type_elements = from_type_tuple.getElements(); + const auto & to_tuple_type_elements = to_type_tuple.getElements(); + + size_t lhs_type_elements_size = from_tuple_type_elements.size(); + if (lhs_type_elements_size != to_tuple_type_elements.size()) + return false; + + for (size_t i = 0; i < lhs_type_elements_size; ++i) + if (!canBeSafelyCasted(from_tuple_type_elements[i], to_tuple_type_elements[i])) + return false; + + return true; + } + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::String: + case TypeIndex::Object: + case TypeIndex::Set: + case TypeIndex::Interval: + case TypeIndex::Function: + case TypeIndex::AggregateFunction: + case TypeIndex::Nothing: + return false; + } + + return true; +} + +} diff --git a/src/DataTypes/Utils.h b/src/DataTypes/Utils.h new file mode 100644 index 00000000000..bee109f4524 --- /dev/null +++ b/src/DataTypes/Utils.h @@ -0,0 +1,19 @@ +#pragma once + +#include + +namespace DB +{ + +/** Returns true if from_type can be safely casted to to_type. + * + * Examples: + * From type UInt8 to type UInt16 returns true. + * From type UInt16 to type UInt8 returns false. + * From type String to type LowCardinality(String) returns true. + * From type LowCardinality(String) to type String returns true. + * From type String to type UInt8 returns false. + */ +bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_type); + +} diff --git a/src/Functions/FunctionHelpers.cpp b/src/Functions/FunctionHelpers.cpp index 7a9817ad344..6d3c20ef2ca 100644 --- a/src/Functions/FunctionHelpers.cpp +++ b/src/Functions/FunctionHelpers.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 47521b9887b..7abd708a944 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -1257,7 +1258,7 @@ bool KeyCondition::tryPrepareSetIndex( if (!future_set) return false; - const auto & set_types = future_set->getTypes(); + const auto set_types = future_set->getTypes(); size_t set_types_size = set_types.size(); size_t indexes_mapping_size = indexes_mapping.size(); @@ -1283,24 +1284,37 @@ bool KeyCondition::tryPrepareSetIndex( for (size_t indexes_mapping_index = 0; indexes_mapping_index < indexes_mapping_size; ++indexes_mapping_index) { + const auto & key_column_type = data_types[indexes_mapping_index]; size_t set_element_index = indexes_mapping[indexes_mapping_index].tuple_index; - const auto & set_element_type = set_types[set_element_index]; - auto & set_column = set_columns[set_element_index]; + auto set_element_type = set_types[set_element_index]; + auto set_column = set_columns[set_element_index]; - bool is_set_column_nullable = set_element_type->isNullable(); - bool is_set_column_low_cardinality_nullable = set_element_type->isLowCardinalityNullable(); + if (canBeSafelyCasted(set_element_type, key_column_type)) + { + set_columns[set_element_index] = castColumn({set_column, set_element_type, {}}, key_column_type); + continue; + } + + if (!key_column_type->canBeInsideNullable()) + return false; const NullMap * set_column_null_map = nullptr; - if (is_set_column_nullable || is_set_column_low_cardinality_nullable) + if (isNullableOrLowCardinalityNullable(set_element_type)) { - if (is_set_column_low_cardinality_nullable) + if (WhichDataType(set_element_type).isLowCardinality()) + { + set_element_type = removeLowCardinality(set_element_type); set_column = set_column->convertToFullColumnIfLowCardinality(); + } - set_column_null_map = &assert_cast(*set_column).getNullMapData(); + set_element_type = removeNullable(set_element_type); + const auto & set_column_nullable = assert_cast(*set_column); + set_column_null_map = &set_column_nullable.getNullMapData(); + set_column = set_column_nullable.getNestedColumnPtr(); } - auto nullable_set_column = castColumnAccurateOrNull({set_column, set_element_type, {}}, data_types[indexes_mapping_index]); + auto nullable_set_column = castColumnAccurateOrNull({set_column, set_element_type, {}}, key_column_type); const auto & nullable_set_column_typed = assert_cast(*nullable_set_column); const auto & nullable_set_column_null_map = nullable_set_column_typed.getNullMapData(); size_t nullable_set_column_null_map_size = nullable_set_column_null_map.size(); @@ -1321,6 +1335,8 @@ bool KeyCondition::tryPrepareSetIndex( set_column = nullable_set_column_typed.getNestedColumn().filter(filter, 0); } + + set_columns[set_element_index] = std::move(set_column); } out.set_index = std::make_shared(set_columns, std::move(indexes_mapping)); From 94d737264cb62016cef3d89dc3c73fce6a5c9fd8 Mon Sep 17 00:00:00 2001 From: priera Date: Thu, 31 Aug 2023 15:37:48 +0200 Subject: [PATCH 60/93] inserting non-duplicate chunks --- src/Processors/Sinks/IOutputChunkGenerator.h | 22 +++++ src/Processors/Sinks/OutputChunkGenerator.cpp | 80 +++++++++++++++++++ src/Processors/Sinks/SinkToStorage.cpp | 13 ++- src/Processors/Sinks/SinkToStorage.h | 7 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 24 ++---- .../MergeTree/ReplicatedMergeTreeSink.h | 12 --- ...erialized_views_duplicated_parts.reference | 14 ++++ ...rt_materialized_views_duplicated_parts.sql | 44 ++++++++++ 8 files changed, 179 insertions(+), 37 deletions(-) create mode 100644 src/Processors/Sinks/IOutputChunkGenerator.h create mode 100644 src/Processors/Sinks/OutputChunkGenerator.cpp create mode 100644 tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.reference create mode 100644 tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.sql diff --git a/src/Processors/Sinks/IOutputChunkGenerator.h b/src/Processors/Sinks/IOutputChunkGenerator.h new file mode 100644 index 00000000000..3a92368df14 --- /dev/null +++ b/src/Processors/Sinks/IOutputChunkGenerator.h @@ -0,0 +1,22 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class IOutputChunkGenerator { +public: + static std::unique_ptr createCopyRanges(ContextPtr context); + static std::unique_ptr createDefault(); + + virtual ~IOutputChunkGenerator() = default; + + virtual void onNewChunkArrived(Chunk chunk) = 0; + virtual void onRowsProcessed(size_t row_count, bool append) = 0; + + virtual Chunk generateChunk() = 0; +}; + +} diff --git a/src/Processors/Sinks/OutputChunkGenerator.cpp b/src/Processors/Sinks/OutputChunkGenerator.cpp new file mode 100644 index 00000000000..dab5adb1792 --- /dev/null +++ b/src/Processors/Sinks/OutputChunkGenerator.cpp @@ -0,0 +1,80 @@ +#include + +namespace DB { + +class ForwardEverythingGenerator : public IOutputChunkGenerator { +public: + + explicit ForwardEverythingGenerator() = default; + + void onNewChunkArrived(Chunk chunk) override + { + in_chunk = chunk.clone(); + } + + void onRowsProcessed(size_t /*row_count*/, bool /*append*/) override + {} + + Chunk generateChunk() override + { + return std::move(in_chunk); + } + +private: + Chunk in_chunk; +}; + +class CopyRangesGenerator : public IOutputChunkGenerator { +public: + explicit CopyRangesGenerator() = default; + + void onNewChunkArrived(Chunk chunk) override + { + out_cols = chunk.cloneEmptyColumns(); + in_chunk = std::move(chunk); + } + + void onRowsProcessed(size_t row_count, bool append) override + { + if (append) + { + const Columns& in_cols = in_chunk.getColumns(); + for (size_t i = 0; i < out_cols.size(); i++) + { + out_cols[i]->insertRangeFrom(*(in_cols[i]), row_offset, row_count); + } + final_chunk_rows += row_count; + } + + row_offset += row_count; + } + + Chunk generateChunk() override + { + return Chunk(std::move(out_cols), final_chunk_rows); + } + +private: + Chunk in_chunk; + MutableColumns out_cols; + size_t row_offset = 0; + size_t final_chunk_rows = 0; +}; + +std::unique_ptr IOutputChunkGenerator::createCopyRanges(ContextPtr context) +{ + // If MV is responsible for deduplication, block is not considered duplicated. + if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) + { + return createDefault(); + } + + return std::make_unique(); +} + +std::unique_ptr IOutputChunkGenerator::createDefault() +{ + return std::make_unique(); +} + +} diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index 5f9f9f9b1a1..84743306446 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -4,7 +4,12 @@ namespace DB { -SinkToStorage::SinkToStorage(const Block & header) : ExceptionKeepingTransform(header, header, false) {} +SinkToStorage::SinkToStorage(const Block & header) : SinkToStorage(header, IOutputChunkGenerator::createDefault()) {} + +SinkToStorage::SinkToStorage(const Block & header, std::unique_ptr output_generator_) + : ExceptionKeepingTransform(header, header, false), + output_generator(std::move(output_generator_)) +{ } void SinkToStorage::onConsume(Chunk chunk) { @@ -15,15 +20,15 @@ void SinkToStorage::onConsume(Chunk chunk) */ Nested::validateArraySizes(getHeader().cloneWithColumns(chunk.getColumns())); + output_generator->onNewChunkArrived(chunk.clone()); consume(chunk.clone()); - if (!lastBlockIsDuplicate()) - cur_chunk = std::move(chunk); } SinkToStorage::GenerateResult SinkToStorage::onGenerate() { GenerateResult res; - res.chunk = std::move(cur_chunk); + + res.chunk = output_generator->generateChunk(); res.is_done = true; return res; } diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index 023bbd8b094..e810578f651 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { @@ -13,13 +14,15 @@ friend class PartitionedSink; public: explicit SinkToStorage(const Block & header); + explicit SinkToStorage(const Block & header, std::unique_ptr output_chunk_generator); const Block & getHeader() const { return inputs.front().getHeader(); } void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } protected: virtual void consume(Chunk chunk) = 0; - virtual bool lastBlockIsDuplicate() const { return false; } + + IOutputChunkGenerator& getOutputGenerator() { return *output_generator; } private: std::vector table_locks; @@ -27,7 +30,7 @@ private: void onConsume(Chunk chunk) override; GenerateResult onGenerate() override; - Chunk cur_chunk; + std::unique_ptr output_generator; }; using SinkToStoragePtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 75679a5750a..9fc7492a7fe 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -130,7 +130,7 @@ ReplicatedMergeTreeSinkImpl::ReplicatedMergeTreeSinkImpl( bool majority_quorum, ContextPtr context_, bool is_attach_) - : SinkToStorage(metadata_snapshot_->getSampleBlock()) + : SinkToStorage(metadata_snapshot_->getSampleBlock(), IOutputChunkGenerator::createCopyRanges(context_)) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , required_quorum_size(majority_quorum ? std::nullopt : std::make_optional(quorum_size)) @@ -386,13 +386,7 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) finishDelayedChunk(zookeeper); delayed_chunk = std::make_unique(); delayed_chunk->partitions = std::move(partitions); - - /// If deduplicated data should not be inserted into MV, we need to set proper - /// value for `last_block_is_duplicate`, which is possible only after the part is committed. - /// Othervide we can delay commit. - /// TODO: we can also delay commit if there is no MVs. - if (!settings.deduplicate_blocks_in_dependent_materialized_views) - finishDelayedChunk(zookeeper); + finishDelayedChunk(zookeeper); ++num_blocks_processed; } @@ -403,8 +397,6 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF if (!delayed_chunk) return; - last_block_is_duplicate = false; - for (auto & partition : delayed_chunk->partitions) { ProfileEventsScope scoped_attach(&partition.part_counters); @@ -415,9 +407,10 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF try { - bool deduplicated = commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num, false).second; + const size_t rowsCount = partition.temp_part.part->rows_count; + const bool deduplicated = commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num, false).second; - last_block_is_duplicate = last_block_is_duplicate || deduplicated; + getOutputGenerator().onRowsProcessed(rowsCount, !deduplicated); /// Set a special error code if the block is duplicate int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; @@ -1092,13 +1085,6 @@ void ReplicatedMergeTreeSinkImpl::onStart() storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event, context, true); } -template -void ReplicatedMergeTreeSinkImpl::onFinish() -{ - auto zookeeper = storage.getZooKeeper(); - finishDelayedChunk(std::make_shared(zookeeper)); -} - template void ReplicatedMergeTreeSinkImpl::waitForQuorum( const ZooKeeperWithFaultInjectionPtr & zookeeper, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 4a192a822f5..b208154631c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -51,23 +51,12 @@ public: void onStart() override; void consume(Chunk chunk) override; - void onFinish() override; String getName() const override { return "ReplicatedMergeTreeSink"; } /// For ATTACHing existing data on filesystem. bool writeExistingPart(MergeTreeData::MutableDataPartPtr & part); - /// For proper deduplication in MaterializedViews - bool lastBlockIsDuplicate() const override - { - /// If MV is responsible for deduplication, block is not considered duplicating. - if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) - return false; - - return last_block_is_duplicate; - } - struct DelayedChunk; private: using BlockIDsType = std::conditional_t, String>; @@ -122,7 +111,6 @@ private: bool is_attach = false; bool quorum_parallel = false; const bool deduplicate = true; - bool last_block_is_duplicate = false; UInt64 num_blocks_processed = 0; using Logger = Poco::Logger; diff --git a/tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.reference b/tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.reference new file mode 100644 index 00000000000..325f639813a --- /dev/null +++ b/tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.reference @@ -0,0 +1,14 @@ +Initial +2020-01-01 13:00:00 24 +Last block is duplicate +2020-01-01 13:00:00 24 +2021-09-01 11:00:00 24 +One block is duplicate (default setting) +2020-01-01 13:00:00 24 +2021-09-01 11:00:00 24 +2022-01-01 12:00:00 24 +One block is duplicate (changed setting) +2020-01-01 13:00:00 24 +2021-09-01 11:00:00 24 +2022-01-01 12:00:00 24 +2023-01-01 12:00:00 24 diff --git a/tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.sql b/tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.sql new file mode 100644 index 00000000000..c087e826a13 --- /dev/null +++ b/tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.sql @@ -0,0 +1,44 @@ +-- Tags: zookeeper + +DROP TABLE IF EXISTS landing SYNC; +DROP TABLE IF EXISTS mv SYNC; + +CREATE TABLE landing +( + `time` DateTime, + `number` Int64 +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{database}/tables/landing/', 'r1') +PARTITION BY toYYYYMMDD(time) +ORDER BY time; + +CREATE MATERIALIZED VIEW mv +ENGINE = ReplicatedSummingMergeTree('/clickhouse/{database}/tables/mv', 'r1') +PARTITION BY toYYYYMMDD(hour) ORDER BY hour +AS SELECT + toStartOfHour(time) AS hour, + sum(number) AS sum_amount + FROM landing GROUP BY hour; + +SELECT 'Initial'; +INSERT INTO landing VALUES ('2020-01-01 13:23:34', 24); +SELECT * FROM mv ORDER BY hour; + +SELECT 'Last block is duplicate'; +INSERT INTO landing VALUES ('2021-09-01 11:00:00', 24), ('2020-01-01 13:23:34', 24); +SELECT * FROM mv ORDER BY hour; + +SELECT 'One block is duplicate (default setting)'; +SET max_insert_delayed_streams_for_parallel_write = 0; +INSERT INTO landing VALUES ('2021-09-01 11:00:00', 24), ('2022-01-01 12:03:00', 24); +SELECT * FROM mv ORDER BY hour; + +SELECT 'One block is duplicate (changed setting)'; +SET max_insert_delayed_streams_for_parallel_write = 5; +INSERT INTO landing VALUES ('2021-09-01 11:00:00', 24), ('2023-01-01 12:03:00', 24); + +SELECT * FROM mv ORDER BY hour; + +DROP TABLE mv; +DROP TABLE landing; + From 6d27ea2742a162351cf1f1738523949a6e7d4ce1 Mon Sep 17 00:00:00 2001 From: priera Date: Tue, 12 Sep 2023 12:49:43 +0200 Subject: [PATCH 61/93] fixed style --- src/Processors/Sinks/IOutputChunkGenerator.h | 3 ++- src/Processors/Sinks/OutputChunkGenerator.cpp | 9 ++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Processors/Sinks/IOutputChunkGenerator.h b/src/Processors/Sinks/IOutputChunkGenerator.h index 3a92368df14..b5e1d76af26 100644 --- a/src/Processors/Sinks/IOutputChunkGenerator.h +++ b/src/Processors/Sinks/IOutputChunkGenerator.h @@ -6,7 +6,8 @@ namespace DB { -class IOutputChunkGenerator { +class IOutputChunkGenerator +{ public: static std::unique_ptr createCopyRanges(ContextPtr context); static std::unique_ptr createDefault(); diff --git a/src/Processors/Sinks/OutputChunkGenerator.cpp b/src/Processors/Sinks/OutputChunkGenerator.cpp index dab5adb1792..eb161c0be1e 100644 --- a/src/Processors/Sinks/OutputChunkGenerator.cpp +++ b/src/Processors/Sinks/OutputChunkGenerator.cpp @@ -1,8 +1,10 @@ #include -namespace DB { +namespace DB +{ -class ForwardEverythingGenerator : public IOutputChunkGenerator { +class ForwardEverythingGenerator : public IOutputChunkGenerator +{ public: explicit ForwardEverythingGenerator() = default; @@ -24,7 +26,8 @@ private: Chunk in_chunk; }; -class CopyRangesGenerator : public IOutputChunkGenerator { +class CopyRangesGenerator : public IOutputChunkGenerator +{ public: explicit CopyRangesGenerator() = default; From b30e33580b54349952b7f52813a3011efbe509a9 Mon Sep 17 00:00:00 2001 From: priera Date: Tue, 12 Sep 2023 17:12:43 +0200 Subject: [PATCH 62/93] fix test --- src/Processors/Sinks/OutputChunkGenerator.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Sinks/OutputChunkGenerator.cpp b/src/Processors/Sinks/OutputChunkGenerator.cpp index eb161c0be1e..107c21b2319 100644 --- a/src/Processors/Sinks/OutputChunkGenerator.cpp +++ b/src/Processors/Sinks/OutputChunkGenerator.cpp @@ -35,6 +35,8 @@ public: { out_cols = chunk.cloneEmptyColumns(); in_chunk = std::move(chunk); + row_offset = 0; + final_chunk_rows = 0; } void onRowsProcessed(size_t row_count, bool append) override From 663ce74609170031d22a330c3478569f7723fea0 Mon Sep 17 00:00:00 2001 From: priera Date: Wed, 13 Sep 2023 14:39:29 +0200 Subject: [PATCH 63/93] fixed clang-tidy error --- src/Processors/Sinks/SinkToStorage.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index e810578f651..dc7ba23b52a 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -14,7 +14,7 @@ friend class PartitionedSink; public: explicit SinkToStorage(const Block & header); - explicit SinkToStorage(const Block & header, std::unique_ptr output_chunk_generator); + explicit SinkToStorage(const Block & header, std::unique_ptr output_generator_); const Block & getHeader() const { return inputs.front().getHeader(); } void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } From b92e37d2dfe0c2088429e1c90fbe73e9f384aa3b Mon Sep 17 00:00:00 2001 From: priera Date: Tue, 19 Sep 2023 12:13:39 +0200 Subject: [PATCH 64/93] Fixing PR comments --- src/Processors/Sinks/IOutputChunkGenerator.h | 5 ++++- src/Processors/Sinks/OutputChunkGenerator.cpp | 20 ++++++++++++------- .../MergeTree/ReplicatedMergeTreeSink.cpp | 3 ++- 3 files changed, 19 insertions(+), 9 deletions(-) diff --git a/src/Processors/Sinks/IOutputChunkGenerator.h b/src/Processors/Sinks/IOutputChunkGenerator.h index b5e1d76af26..824313a2394 100644 --- a/src/Processors/Sinks/IOutputChunkGenerator.h +++ b/src/Processors/Sinks/IOutputChunkGenerator.h @@ -6,10 +6,13 @@ namespace DB { +/// This interface is meant to be used by the SinkToStorage processor +/// SinkToStorage delegates on it the creation of the data chunk that will deliver to the next stages of the query pipeline +/// Default implementation (createDefault() factory method) just forwards everything that it receives class IOutputChunkGenerator { public: - static std::unique_ptr createCopyRanges(ContextPtr context); + static std::unique_ptr createCopyRanges(bool deduplicate_later); static std::unique_ptr createDefault(); virtual ~IOutputChunkGenerator() = default; diff --git a/src/Processors/Sinks/OutputChunkGenerator.cpp b/src/Processors/Sinks/OutputChunkGenerator.cpp index 107c21b2319..942bf49a2d4 100644 --- a/src/Processors/Sinks/OutputChunkGenerator.cpp +++ b/src/Processors/Sinks/OutputChunkGenerator.cpp @@ -3,6 +3,7 @@ namespace DB { +/// Default implementation. The new chunk received is forwarded as-is to the next stages of the query class ForwardEverythingGenerator : public IOutputChunkGenerator { public: @@ -14,10 +15,10 @@ public: in_chunk = chunk.clone(); } - void onRowsProcessed(size_t /*row_count*/, bool /*append*/) override + void onRowsProcessed(size_t /*row_count*/, bool /*append*/) override {} - Chunk generateChunk() override + Chunk generateChunk() override { return std::move(in_chunk); } @@ -26,6 +27,10 @@ private: Chunk in_chunk; }; +/// Specific implementation which generates a chunk with just a subset of the rows received originally +/// Rows are assumed to be processed in the same order than they appear in the original chunk +/// Is up to the client to decide how many rows process at once, but after each range processed, +/// onRowsProcessed() has to be called, indicating whether append that range to the output chunk or not class CopyRangesGenerator : public IOutputChunkGenerator { public: @@ -39,7 +44,7 @@ public: final_chunk_rows = 0; } - void onRowsProcessed(size_t row_count, bool append) override + void onRowsProcessed(size_t row_count, bool append) override { if (append) { @@ -54,7 +59,7 @@ public: row_offset += row_count; } - Chunk generateChunk() override + Chunk generateChunk() override { return Chunk(std::move(out_cols), final_chunk_rows); } @@ -66,10 +71,11 @@ private: size_t final_chunk_rows = 0; }; -std::unique_ptr IOutputChunkGenerator::createCopyRanges(ContextPtr context) +std::unique_ptr IOutputChunkGenerator::createCopyRanges(bool deduplicate_later) { - // If MV is responsible for deduplication, block is not considered duplicated. - if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) + // If MV is responsible for deduplication, block won't be considered duplicated. + // So default implementation, forwarding all the data, is used + if (deduplicate_later) { return createDefault(); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 9fc7492a7fe..9cc1c6932c7 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -130,7 +130,8 @@ ReplicatedMergeTreeSinkImpl::ReplicatedMergeTreeSinkImpl( bool majority_quorum, ContextPtr context_, bool is_attach_) - : SinkToStorage(metadata_snapshot_->getSampleBlock(), IOutputChunkGenerator::createCopyRanges(context_)) + : SinkToStorage(metadata_snapshot_->getSampleBlock(), + IOutputChunkGenerator::createCopyRanges(context_->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views)) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , required_quorum_size(majority_quorum ? std::nullopt : std::make_optional(quorum_size)) From 29e83fbe6494de34d7c2b3c41569cc38f9385515 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 19 Sep 2023 10:24:42 +0000 Subject: [PATCH 65/93] Docs: Update query cache docs about usage in view definitions Cf. https://github.com/ClickHouse/support-escalation/issues/963 --- docs/en/operations/query-cache.md | 10 ++++++++-- docs/en/operations/settings/settings.md | 2 +- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 137ebf8ad85..bbde77338af 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -43,6 +43,12 @@ SETTINGS use_query_cache = true; will store the query result in the query cache. Subsequent executions of the same query (also with parameter `use_query_cache = true`) will read the computed result from the cache and return it immediately. +:::note +Setting `use_query_cache` and all other query-cache-related settings only take an effect on stand-alone `SELECT` statements. In particular, +the results of `SELECT`s to views created by `CREATE VIEW AS SELECT [...] SETTINGS use_query_cache = true` are not cached unless the `SELECT` +statement runs with `SETTINGS use_query_cache = true`. +::: + The way the cache is utilized can be configured in more detail using settings [enable_writes_to_query_cache](settings/settings.md#enable-writes-to-query-cache) and [enable_reads_from_query_cache](settings/settings.md#enable-reads-from-query-cache) (both `true` by default). The former setting controls whether query results are stored in the cache, whereas the latter setting determines if the database should try to retrieve query @@ -84,7 +90,7 @@ It is also possible to limit the cache usage of individual users using [settings constraints](settings/constraints-on-settings.md). More specifically, you can restrict the maximum amount of memory (in bytes) a user may allocate in the query cache and the the maximum number of stored query results. For that, first provide configurations [query_cache_max_size_in_bytes](settings/settings.md#query-cache-max-size-in-bytes) and -[query_cache_max_entries](settings/settings.md#query-cache-size-max-items) in a user profile in `users.xml`, then make both settings +[query_cache_max_entries](settings/settings.md#query-cache-size-max-entries) in a user profile in `users.xml`, then make both settings readonly: ``` xml @@ -134,7 +140,7 @@ block granularity when query results are later served from the query cache. As a result, the query cache stores for each query multiple (partial) result blocks. While this behavior is a good default, it can be suppressed using setting -[query_cache_squash_partial_query_results](settings/settings.md#query-cache-squash-partial-query-results). +[query_cache_squash_partial_results](settings/settings.md#query-cache-squash-partial-results). Also, results of queries with non-deterministic functions are not cached. Such functions include - functions for accessing dictionaries: [`dictGet()`](../sql-reference/functions/ext-dict-functions.md#dictGet) etc. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e9e5920fa59..1e9abb500f8 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1637,7 +1637,7 @@ Possible values: Default value: `1`. -## query_cache_store_results_of_queries_with_nondeterministic_functions {#query--store-results-of-queries-with-nondeterministic-functions} +## query_cache_store_results_of_queries_with_nondeterministic_functions {#query-cache-store-results-of-queries-with-nondeterministic-functions} If turned on, then results of `SELECT` queries with non-deterministic functions (e.g. `rand()`, `now()`) can be cached in the [query cache](../query-cache.md). From 2e8e4b57665785f223134f39dba8e63599c204f0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 19 Sep 2023 12:51:34 +0200 Subject: [PATCH 66/93] Fix broken tests for clickhouse-diagnostics --- programs/diagnostics/internal/platform/data/file_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/diagnostics/internal/platform/data/file_test.go b/programs/diagnostics/internal/platform/data/file_test.go index 5df1f8cc359..bb6397b82e8 100644 --- a/programs/diagnostics/internal/platform/data/file_test.go +++ b/programs/diagnostics/internal/platform/data/file_test.go @@ -135,7 +135,7 @@ func TestConfigFileFrameCopy(t *testing.T) { sizes := map[string]int64{ "users.xml": int64(2017), "default-password.xml": int64(188), - "config.xml": int64(59506), + "config.xml": int64(59377), "server-include.xml": int64(168), "user-include.xml": int64(559), } From 363ee7cae58c4118452bd856406d55469dae9e94 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Sep 2023 13:41:55 +0200 Subject: [PATCH 67/93] Better check for exceptions in checkData of non replicated merge tree --- src/Storages/MergeTree/checkDataPart.cpp | 46 ++++++++++++++++++++++++ src/Storages/MergeTree/checkDataPart.h | 2 +- src/Storages/StorageMergeTree.cpp | 14 +++++--- 3 files changed, 57 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 1717d91271a..6d8e02701fa 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -15,6 +15,11 @@ #include #include #include +#include + +#if USE_AZURE_BLOB_STORAGE +# include +#endif namespace CurrentMetrics { @@ -68,6 +73,47 @@ bool isRetryableException(const Exception & e) return false; } +bool isRetryableException(const std::exception_ptr exception_ptr) +{ + try + { + rethrow_exception(exception_ptr); + } +#if USE_AWS_S3 + catch (const S3Exception & s3_exception) + { + if (s3_exception.isRetryableError()) + return true; + } +#endif +#if USE_AZURE_BLOB_STORAGE + catch (const Azure::Core::RequestFailedException &) + { + return true; + } +#endif + catch (const Exception & e) + { + if (isNotEnoughMemoryErrorCode(e.code())) + return true; + + if (e.code() == ErrorCodes::NETWORK_ERROR || e.code() == ErrorCodes::SOCKET_TIMEOUT) + return true; + } + catch (const Poco::Net::NetException &) + { + return true; + } + catch (const Poco::TimeoutException &) + { + return true; + } + + /// In fact, there can be other similar situations. + /// But it is OK, because there is a safety guard against deleting too many parts. + return false; +} + static IMergeTreeDataPart::Checksums checkDataPart( MergeTreeData::DataPartPtr data_part, diff --git a/src/Storages/MergeTree/checkDataPart.h b/src/Storages/MergeTree/checkDataPart.h index 20ddecad3ed..88e86db37fc 100644 --- a/src/Storages/MergeTree/checkDataPart.h +++ b/src/Storages/MergeTree/checkDataPart.h @@ -14,5 +14,5 @@ IMergeTreeDataPart::Checksums checkDataPart( bool isNotEnoughMemoryErrorCode(int code); bool isRetryableException(const Exception & e); - +bool isRetryableException(const std::exception_ptr exception_ptr); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 4f27cbc76ed..b2e6e6d33fa 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2222,10 +2222,13 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ results.emplace_back(part->name, true, "Checksums recounted and written to disk."); } - catch (const Exception & ex) + catch (...) { + if (isRetryableException(std::current_exception())) + throw; + tryLogCurrentException(log, __PRETTY_FUNCTION__); - results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + results.emplace_back(part->name, false, "Check of part finished with error: '" + getCurrentExceptionMessage(false) + "'"); } } else @@ -2235,9 +2238,12 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ checkDataPart(part, true); results.emplace_back(part->name, true, ""); } - catch (const Exception & ex) + catch (...) { - results.emplace_back(part->name, false, ex.message()); + if (isRetryableException(std::current_exception())) + throw; + + results.emplace_back(part->name, false, getCurrentExceptionMessage(false)); } } } From 7b1cf9d912c00eb26f9d368f2fcc9742a74249eb Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Sep 2023 13:45:34 +0200 Subject: [PATCH 68/93] Revert "Better check for exceptions in checkData of non replicated merge tree" This reverts commit 363ee7cae58c4118452bd856406d55469dae9e94. --- src/Storages/MergeTree/checkDataPart.cpp | 46 ------------------------ src/Storages/MergeTree/checkDataPart.h | 2 +- src/Storages/StorageMergeTree.cpp | 14 +++----- 3 files changed, 5 insertions(+), 57 deletions(-) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 6d8e02701fa..1717d91271a 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -15,11 +15,6 @@ #include #include #include -#include - -#if USE_AZURE_BLOB_STORAGE -# include -#endif namespace CurrentMetrics { @@ -73,47 +68,6 @@ bool isRetryableException(const Exception & e) return false; } -bool isRetryableException(const std::exception_ptr exception_ptr) -{ - try - { - rethrow_exception(exception_ptr); - } -#if USE_AWS_S3 - catch (const S3Exception & s3_exception) - { - if (s3_exception.isRetryableError()) - return true; - } -#endif -#if USE_AZURE_BLOB_STORAGE - catch (const Azure::Core::RequestFailedException &) - { - return true; - } -#endif - catch (const Exception & e) - { - if (isNotEnoughMemoryErrorCode(e.code())) - return true; - - if (e.code() == ErrorCodes::NETWORK_ERROR || e.code() == ErrorCodes::SOCKET_TIMEOUT) - return true; - } - catch (const Poco::Net::NetException &) - { - return true; - } - catch (const Poco::TimeoutException &) - { - return true; - } - - /// In fact, there can be other similar situations. - /// But it is OK, because there is a safety guard against deleting too many parts. - return false; -} - static IMergeTreeDataPart::Checksums checkDataPart( MergeTreeData::DataPartPtr data_part, diff --git a/src/Storages/MergeTree/checkDataPart.h b/src/Storages/MergeTree/checkDataPart.h index 88e86db37fc..20ddecad3ed 100644 --- a/src/Storages/MergeTree/checkDataPart.h +++ b/src/Storages/MergeTree/checkDataPart.h @@ -14,5 +14,5 @@ IMergeTreeDataPart::Checksums checkDataPart( bool isNotEnoughMemoryErrorCode(int code); bool isRetryableException(const Exception & e); -bool isRetryableException(const std::exception_ptr exception_ptr); + } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b2e6e6d33fa..4f27cbc76ed 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2222,13 +2222,10 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ results.emplace_back(part->name, true, "Checksums recounted and written to disk."); } - catch (...) + catch (const Exception & ex) { - if (isRetryableException(std::current_exception())) - throw; - tryLogCurrentException(log, __PRETTY_FUNCTION__); - results.emplace_back(part->name, false, "Check of part finished with error: '" + getCurrentExceptionMessage(false) + "'"); + results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); } } else @@ -2238,12 +2235,9 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ checkDataPart(part, true); results.emplace_back(part->name, true, ""); } - catch (...) + catch (const Exception & ex) { - if (isRetryableException(std::current_exception())) - throw; - - results.emplace_back(part->name, false, getCurrentExceptionMessage(false)); + results.emplace_back(part->name, false, ex.message()); } } } From 9dcc41cf83d71f014c7e612c9653c1c789ca6d34 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Sep 2023 13:47:32 +0200 Subject: [PATCH 69/93] Fxi --- src/Storages/StorageMergeTree.cpp | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index bae91ec8bb1..22700712829 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2231,10 +2231,13 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ results.emplace_back(part->name, true, "Checksums recounted and written to disk."); } - catch (const Exception & ex) + catch (...) { + if (isRetryableException(std::current_exception())) + throw; + tryLogCurrentException(log, __PRETTY_FUNCTION__); - results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + results.emplace_back(part->name, false, "Check of part finished with error: '" + getCurrentExceptionMessage(false) + "'"); } } else @@ -2244,9 +2247,12 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ checkDataPart(part, true); results.emplace_back(part->name, true, ""); } - catch (const Exception & ex) + catch (...) { - results.emplace_back(part->name, false, ex.message()); + if (isRetryableException(std::current_exception())) + throw; + + results.emplace_back(part->name, false, getCurrentExceptionMessage(false)); } } } From 8c29408f5eba3388a094fa937d19c251a5770791 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 7 Sep 2023 14:06:19 +0000 Subject: [PATCH 70/93] Parse data in JSON format as JSONEachRow if failed to parse metadata --- src/Formats/JSONUtils.cpp | 17 ++-- src/Formats/JSONUtils.h | 1 + .../Formats/Impl/JSONEachRowRowInputFormat.h | 3 +- .../Formats/Impl/JSONRowInputFormat.cpp | 82 ++++++++++++++----- .../Formats/Impl/JSONRowInputFormat.h | 18 +++- ..._as_json_each_row_on_no_metadata.reference | 3 + ...e_json_as_json_each_row_on_no_metadata.sql | 3 + 7 files changed, 98 insertions(+), 29 deletions(-) create mode 100644 tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.reference create mode 100644 tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.sql diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 6fbda869154..1a2849493c3 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -687,10 +687,9 @@ namespace JSONUtils return names_and_types; } - NamesAndTypesList readMetadataAndValidateHeader(ReadBuffer & in, const Block & header) + void validateMetadataByHeader(const NamesAndTypesList & names_and_types_from_metadata, const Block & header) { - auto names_and_types = JSONUtils::readMetadata(in); - for (const auto & [name, type] : names_and_types) + for (const auto & [name, type] : names_and_types_from_metadata) { if (!header.has(name)) continue; @@ -698,10 +697,16 @@ namespace JSONUtils auto header_type = header.getByName(name).type; if (!type->equals(*header_type)) throw Exception( - ErrorCodes::INCORRECT_DATA, - "Type {} of column '{}' from metadata is not the same as type in header {}", - type->getName(), name, header_type->getName()); + ErrorCodes::INCORRECT_DATA, + "Type {} of column '{}' from metadata is not the same as type in header {}", + type->getName(), name, header_type->getName()); } + } + + NamesAndTypesList readMetadataAndValidateHeader(ReadBuffer & in, const Block & header) + { + auto names_and_types = JSONUtils::readMetadata(in); + validateMetadataByHeader(names_and_types, header); return names_and_types; } diff --git a/src/Formats/JSONUtils.h b/src/Formats/JSONUtils.h index bd56eb646cb..27a357447f5 100644 --- a/src/Formats/JSONUtils.h +++ b/src/Formats/JSONUtils.h @@ -124,6 +124,7 @@ namespace JSONUtils NamesAndTypesList readMetadata(ReadBuffer & in); NamesAndTypesList readMetadataAndValidateHeader(ReadBuffer & in, const Block & header); + void validateMetadataByHeader(const NamesAndTypesList & names_and_types_from_metadata, const Block & header); bool skipUntilFieldInObject(ReadBuffer & in, const String & desired_field_name); void skipTheRestOfObject(ReadBuffer & in); diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h index d4246c37ea0..ad494d07fbc 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h @@ -32,10 +32,11 @@ public: String getName() const override { return "JSONEachRowRowInputFormat"; } void resetParser() override; -private: +protected: void readPrefix() override; void readSuffix() override; +private: bool readRow(MutableColumns & columns, RowReadExtension & ext) override; bool allowSyncAfterError() const override { return true; } void syncAfterError() override; diff --git a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp index ac7ba6048a5..a1507050fbc 100644 --- a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp @@ -12,42 +12,84 @@ namespace ErrorCodes } JSONRowInputFormat::JSONRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) - : JSONEachRowRowInputFormat(in_, header_, params_, format_settings_, false), validate_types_from_metadata(format_settings_.json.validate_types_from_metadata) + : JSONRowInputFormat(std::make_unique(in_), header_, params_, format_settings_) +{ +} + +JSONRowInputFormat::JSONRowInputFormat(std::unique_ptr buf, const DB::Block & header_, DB::IRowInputFormat::Params params_, const DB::FormatSettings & format_settings_) + : JSONEachRowRowInputFormat(*buf, header_, params_, format_settings_, false), validate_types_from_metadata(format_settings_.json.validate_types_from_metadata), peekable_buf(std::move(buf)) { } void JSONRowInputFormat::readPrefix() { - skipBOMIfExists(*in); - JSONUtils::skipObjectStart(*in); + skipBOMIfExists(*peekable_buf); + + PeekableReadBufferCheckpoint checkpoint(*peekable_buf); + NamesAndTypesList names_and_types_from_metadata; + + /// Try to parse metadata, if failed, try to parse data as JSONEachRow format. + try + { + JSONUtils::skipObjectStart(*peekable_buf); + names_and_types_from_metadata = JSONUtils::readMetadata(*peekable_buf); + JSONUtils::skipComma(*peekable_buf); + if (!JSONUtils::skipUntilFieldInObject(*peekable_buf, "data")) + throw Exception(ErrorCodes::INCORRECT_DATA, "Expected field \"data\" with table content"); + + JSONUtils::skipArrayStart(*peekable_buf); + data_in_square_brackets = true; + } + catch (...) + { + peekable_buf->rollbackToCheckpoint(); + JSONEachRowRowInputFormat::readPrefix(); + parse_as_json_each_row = true; + return; + } + if (validate_types_from_metadata) - JSONUtils::readMetadataAndValidateHeader(*in, getPort().getHeader()); - else - JSONUtils::readMetadata(*in); - - JSONUtils::skipComma(*in); - if (!JSONUtils::skipUntilFieldInObject(*in, "data")) - throw Exception(ErrorCodes::INCORRECT_DATA, "Expected field \"data\" with table content"); - - JSONUtils::skipArrayStart(*in); - data_in_square_brackets = true; + JSONUtils::validateMetadataByHeader(names_and_types_from_metadata, getPort().getHeader()); } void JSONRowInputFormat::readSuffix() { - JSONUtils::skipArrayEnd(*in); - JSONUtils::skipTheRestOfObject(*in); + if (parse_as_json_each_row) + { + JSONEachRowRowInputFormat::readSuffix(); + } + else + { + JSONUtils::skipArrayEnd(*peekable_buf); + JSONUtils::skipTheRestOfObject(*peekable_buf); + } } -JSONRowSchemaReader::JSONRowSchemaReader(ReadBuffer & in_) : ISchemaReader(in_) +JSONRowSchemaReader::JSONRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) + : JSONRowSchemaReader(std::make_unique(in_), format_settings_) +{ +} + +JSONRowSchemaReader::JSONRowSchemaReader(std::unique_ptr buf, const DB::FormatSettings & format_settings_) + : JSONEachRowSchemaReader(*buf, format_settings_), peekable_buf(std::move(buf)) { } NamesAndTypesList JSONRowSchemaReader::readSchema() { - skipBOMIfExists(in); - JSONUtils::skipObjectStart(in); - return JSONUtils::readMetadata(in); + skipBOMIfExists(*peekable_buf); + PeekableReadBufferCheckpoint checkpoint(*peekable_buf); + /// Try to parse metadata, if failed, try to parse data as JSONEachRow format + try + { + JSONUtils::skipObjectStart(*peekable_buf); + return JSONUtils::readMetadata(*peekable_buf); + } + catch (...) + { + peekable_buf->rollbackToCheckpoint(true); + return JSONEachRowSchemaReader::readSchema(); + } } void registerInputFormatJSON(FormatFactory & factory) @@ -69,7 +111,7 @@ void registerJSONSchemaReader(FormatFactory & factory) auto register_schema_reader = [&](const String & format) { factory.registerSchemaReader( - format, [](ReadBuffer & buf, const FormatSettings &) { return std::make_unique(buf); }); + format, [](ReadBuffer & buf, const FormatSettings & format_settings) { return std::make_unique(buf, format_settings); }); }; register_schema_reader("JSON"); /// JSONCompact has the same suffix with metadata. diff --git a/src/Processors/Formats/Impl/JSONRowInputFormat.h b/src/Processors/Formats/Impl/JSONRowInputFormat.h index 40b7cc2a268..5bdffe3dec0 100644 --- a/src/Processors/Formats/Impl/JSONRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowInputFormat.h @@ -24,20 +24,34 @@ public: String getName() const override { return "JSONRowInputFormat"; } private: + JSONRowInputFormat( + std::unique_ptr buf, + const Block & header_, + Params params_, + const FormatSettings & format_settings_); + void readPrefix() override; void readSuffix() override; const bool validate_types_from_metadata; + bool parse_as_json_each_row = false; + std::unique_ptr peekable_buf; + std::exception_ptr reading_metadata_exception; }; -class JSONRowSchemaReader : public ISchemaReader +class JSONRowSchemaReader : public JSONEachRowSchemaReader { public: - JSONRowSchemaReader(ReadBuffer & in_); + JSONRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_); NamesAndTypesList readSchema() override; bool hasStrictOrderOfColumns() const override { return false; } + +private: + JSONRowSchemaReader(std::unique_ptr buf, const FormatSettings & format_settings_); + + std::unique_ptr peekable_buf; }; } diff --git a/tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.reference b/tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.reference new file mode 100644 index 00000000000..9c61868a7dd --- /dev/null +++ b/tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.reference @@ -0,0 +1,3 @@ +a Nullable(Int64) +b Nullable(String) +10 Hello diff --git a/tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.sql b/tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.sql new file mode 100644 index 00000000000..ba7b720e3b6 --- /dev/null +++ b/tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.sql @@ -0,0 +1,3 @@ +desc format(JSON, '{"a" : 10, "b" : "Hello"}'); +select * from format(JSON, '{"a" : 10, "b" : "Hello"}'); + From 5bd2e9f61043a30cb24f1f4221c50546d3426192 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 7 Sep 2023 16:57:25 +0000 Subject: [PATCH 71/93] Fix tests --- src/Processors/Formats/Impl/JSONRowInputFormat.cpp | 11 +++++++++++ src/Processors/Formats/Impl/JSONRowInputFormat.h | 3 +++ 2 files changed, 14 insertions(+) diff --git a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp index a1507050fbc..745c510be1a 100644 --- a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp @@ -65,6 +65,17 @@ void JSONRowInputFormat::readSuffix() } } +void JSONRowInputFormat::setReadBuffer(DB::ReadBuffer & in_) +{ + peekable_buf->setSubBuffer(in_); +} + +void JSONRowInputFormat::resetParser() +{ + JSONEachRowRowInputFormat::resetParser(); + peekable_buf->reset(); +} + JSONRowSchemaReader::JSONRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) : JSONRowSchemaReader(std::make_unique(in_), format_settings_) { diff --git a/src/Processors/Formats/Impl/JSONRowInputFormat.h b/src/Processors/Formats/Impl/JSONRowInputFormat.h index 5bdffe3dec0..771c0b64aee 100644 --- a/src/Processors/Formats/Impl/JSONRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowInputFormat.h @@ -23,6 +23,9 @@ public: String getName() const override { return "JSONRowInputFormat"; } + void setReadBuffer(ReadBuffer & in_) override; + void resetParser() override; + private: JSONRowInputFormat( std::unique_ptr buf, From f974970c3cb88474f053ce59808926e8c2f28a76 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 18 Sep 2023 18:41:04 +0000 Subject: [PATCH 72/93] Apply suggestion --- .../Formats/Impl/JSONRowInputFormat.cpp | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp index 745c510be1a..90537061723 100644 --- a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp @@ -40,16 +40,27 @@ void JSONRowInputFormat::readPrefix() JSONUtils::skipArrayStart(*peekable_buf); data_in_square_brackets = true; } - catch (...) + catch (const ParsingException &) + { + parse_as_json_each_row = true; + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::INCORRECT_DATA) + throw; + + parse_as_json_each_row = true; + } + + if (parse_as_json_each_row) { peekable_buf->rollbackToCheckpoint(); JSONEachRowRowInputFormat::readPrefix(); - parse_as_json_each_row = true; - return; } - - if (validate_types_from_metadata) + else if (validate_types_from_metadata) + { JSONUtils::validateMetadataByHeader(names_and_types_from_metadata, getPort().getHeader()); + } } void JSONRowInputFormat::readSuffix() From 003f5f77eb22731b86a316a2ab133fbfb920a84a Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 19 Sep 2023 15:01:28 +0200 Subject: [PATCH 73/93] refine error code of duplicated index in create query --- src/Interpreters/InterpreterCreateQuery.cpp | 4 ++++ .../0_stateless/02884_duplicate_index_name.reference | 0 .../queries/0_stateless/02884_duplicate_index_name.sql | 10 ++++++++++ 3 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/02884_duplicate_index_name.reference create mode 100644 tests/queries/0_stateless/02884_duplicate_index_name.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 1cd34c2a0f6..b426025413c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -95,6 +95,7 @@ namespace ErrorCodes extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY; extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE; extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_INDEX; extern const int LOGICAL_ERROR; extern const int UNKNOWN_DATABASE; extern const int PATH_ACCESS_DENIED; @@ -697,6 +698,8 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti for (const auto & index : create.columns_list->indices->children) { IndexDescription index_desc = IndexDescription::getIndexFromAST(index->clone(), properties.columns, getContext()); + if (properties.indices.has(index_desc.name)) + throw Exception(ErrorCodes::ILLEGAL_INDEX, "Duplicated index name {}", backQuoteIfNeed(index_desc.name)); const auto & settings = getContext()->getSettingsRef(); if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index) { @@ -711,6 +714,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti properties.indices.push_back(index_desc); } + if (create.columns_list->projections) for (const auto & projection_ast : create.columns_list->projections->children) { diff --git a/tests/queries/0_stateless/02884_duplicate_index_name.reference b/tests/queries/0_stateless/02884_duplicate_index_name.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02884_duplicate_index_name.sql b/tests/queries/0_stateless/02884_duplicate_index_name.sql new file mode 100644 index 00000000000..4647ab3702b --- /dev/null +++ b/tests/queries/0_stateless/02884_duplicate_index_name.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS test_dup_index; + +CREATE TABLE test_dup_index +( + a Int64, + b Int64, + INDEX idx_a a TYPE minmax, + INDEX idx_a b TYPE minmax +) Engine = MergeTree() +ORDER BY a; -- { serverError 127 } From 3e4eefc700d48284cd3c0bf788914dbcf7fd559a Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 19 Sep 2023 15:56:46 +0200 Subject: [PATCH 74/93] Update tests/queries/0_stateless/02884_duplicate_index_name.sql Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- tests/queries/0_stateless/02884_duplicate_index_name.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02884_duplicate_index_name.sql b/tests/queries/0_stateless/02884_duplicate_index_name.sql index 4647ab3702b..4cd9ae6d2a2 100644 --- a/tests/queries/0_stateless/02884_duplicate_index_name.sql +++ b/tests/queries/0_stateless/02884_duplicate_index_name.sql @@ -7,4 +7,4 @@ CREATE TABLE test_dup_index INDEX idx_a a TYPE minmax, INDEX idx_a b TYPE minmax ) Engine = MergeTree() -ORDER BY a; -- { serverError 127 } +ORDER BY a; -- { serverError ILLEGAL_INDEX } From 27d8eefe9376ba73ee8a898ec674d324b1a91d7c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 19 Sep 2023 17:14:32 +0300 Subject: [PATCH 75/93] Fixed tests --- src/DataTypes/Utils.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/Utils.cpp b/src/DataTypes/Utils.cpp index 0168c6d256c..1c36b2dfc9d 100644 --- a/src/DataTypes/Utils.cpp +++ b/src/DataTypes/Utils.cpp @@ -15,10 +15,10 @@ bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_typ bool to_type_was_nullable = isNullableOrLowCardinalityNullable(to_type); auto to_type_unwrapped = removeNullable(removeLowCardinality(to_type)); - if (from_type->equals(*to_type)) + if (from_type->equals(*to_type_unwrapped)) return true; - auto to_which_type = WhichDataType(to_type->getTypeId()); + auto to_which_type = WhichDataType(to_type_unwrapped->getTypeId()); switch (from_which_type.idx) { From 51b95abc2b9f4e8577d4376129b5c0c7f6b3fbca Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 19 Sep 2023 17:19:11 +0200 Subject: [PATCH 76/93] Update Settings.h (#54754) --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7a20c685b09..186d90d9f5d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -796,7 +796,7 @@ class IColumn; M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \ M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \ M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ - M(Bool, allow_experimental_undrop_table_query, false, "Allow to use undrop query to restore dropped table in a limited time", 0) \ + M(Bool, allow_experimental_undrop_table_query, true, "Allow to use undrop query to restore dropped table in a limited time", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ From b1b49f430af91868d9da8cb0a34aa5fa58e0bc3b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 19 Sep 2023 15:32:58 +0000 Subject: [PATCH 77/93] review suggestions --- docker/images.json | 1 + tests/ci/libfuzzer_test_check.py | 10 ++++------ 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/docker/images.json b/docker/images.json index bddfd49ea3b..d208ee9a888 100644 --- a/docker/images.json +++ b/docker/images.json @@ -125,6 +125,7 @@ "name": "clickhouse/test-base", "dependent": [ "docker/test/fuzzer", + "docker/test/libfuzzer", "docker/test/integration/base", "docker/test/keeper-jepsen", "docker/test/server-jepsen", diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 8d307b22042..e768b7f1b4e 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -139,9 +139,8 @@ def main(): docker_image = get_image_with_version(reports_path, "clickhouse/libfuzzer") - fuzzers_path = Path(temp_path) / "fuzzers" - if not fuzzers_path.exists(): - os.makedirs(fuzzers_path) + fuzzers_path = temp_path / "fuzzers" + fuzzers_path.mkdir(parents=True, exist_ok=True) download_fuzzers(check_name, reports_path, fuzzers_path) @@ -152,9 +151,8 @@ def main(): corpus_path = fuzzers_path / (file.removesuffix("_seed_corpus.zip") + ".in") zipfile.ZipFile(fuzzers_path / file, "r").extractall(corpus_path) - result_path = Path(temp_path) / "result_path" - if not result_path.exists(): - os.makedirs(result_path) + result_path = temp_path / "result_path" + result_path.mkdir(parents=True, exist_ok=True) run_log_path = result_path / "run.log" From a15029ecb593e218cabeb3ae2af5d2afa3f22c6e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 19 Sep 2023 16:29:02 +0000 Subject: [PATCH 78/93] Query Cache: Reject queries with non-deterministic functions by default https://github.com/ClickHouse/support-escalation/issues/963 --- docs/en/operations/query-cache.md | 4 ++-- src/Common/ErrorCodes.cpp | 1 + src/Interpreters/executeQuery.cpp | 14 +++++++++----- ...uery_cache_nondeterministic_functions.reference | 1 - ...2494_query_cache_nondeterministic_functions.sql | 6 +++--- 5 files changed, 15 insertions(+), 11 deletions(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index bbde77338af..e111206355e 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -142,7 +142,7 @@ As a result, the query cache stores for each query multiple (partial) result blocks. While this behavior is a good default, it can be suppressed using setting [query_cache_squash_partial_results](settings/settings.md#query-cache-squash-partial-results). -Also, results of queries with non-deterministic functions are not cached. Such functions include +Also, results of queries with non-deterministic functions are not cached by default. Such functions include - functions for accessing dictionaries: [`dictGet()`](../sql-reference/functions/ext-dict-functions.md#dictGet) etc. - [user-defined functions](../sql-reference/statements/create/function.md), - functions which return the current date or time: [`now()`](../sql-reference/functions/date-time-functions.md#now), @@ -158,7 +158,7 @@ Also, results of queries with non-deterministic functions are not cached. Such f - functions which depend on the environment: [`currentUser()`](../sql-reference/functions/other-functions.md#currentUser), [`queryID()`](../sql-reference/functions/other-functions.md#queryID), [`getMacro()`](../sql-reference/functions/other-functions.md#getMacro) etc. -Caching of non-deterministic functions can be forced regardless using setting +To force caching of results of queries with non-deterministic functionsregardless, using setting [query_cache_store_results_of_queries_with_nondeterministic_functions](settings/settings.md#query-cache-store-results-of-queries-with-nondeterministic-functions). Finally, entries in the query cache are not shared between users due to security reasons. For example, user A must not be able to bypass a diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f23685c37d1..ad34516b00e 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -585,6 +585,7 @@ M(700, USER_SESSION_LIMIT_EXCEEDED) \ M(701, CLUSTER_DOESNT_EXIST) \ M(702, CLIENT_INFO_DOES_NOT_MATCH) \ + M(703, CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 310af2f9812..9b24b5df9b2 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -94,11 +94,12 @@ namespace DB namespace ErrorCodes { + extern const int CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS; extern const int INTO_OUTFILE_NOT_ALLOWED; - extern const int QUERY_WAS_CANCELLED; extern const int INVALID_TRANSACTION; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; + extern const int QUERY_WAS_CANCELLED; } @@ -991,7 +992,7 @@ static std::tuple executeQueryImpl( if (!async_insert) { - /// If it is a non-internal SELECT, and passive/read use of the query cache is enabled, and the cache knows the query, then set + /// If it is a non-internal SELECT, and passive (read) use of the query cache is enabled, and the cache knows the query, then set /// a pipeline with a source populated by the query cache. auto get_result_from_query_cache = [&]() { @@ -1091,11 +1092,14 @@ static std::tuple executeQueryImpl( res = interpreter->execute(); - /// If it is a non-internal SELECT query, and active/write use of the query cache is enabled, then add a processor on + /// If it is a non-internal SELECT query, and active (write) use of the query cache is enabled, then add a processor on /// top of the pipeline which stores the result in the query cache. - if (can_use_query_cache && settings.enable_writes_to_query_cache - && (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions)) + if (can_use_query_cache && settings.enable_writes_to_query_cache) { + if (astContainsNonDeterministicFunctions(ast, context) && !settings.query_cache_store_results_of_queries_with_nondeterministic_functions) + throw Exception(ErrorCodes::CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS, + "Unable to cache the query result because the query contains a non-deterministic function. Use setting query_cache_store_results_of_queries_with_nondeterministic_functions = 1 to store the query result regardless."); + QueryCache::Key key( ast, res.pipeline.getHeader(), context->getUserName(), settings.query_cache_share_between_users, diff --git a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.reference b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.reference index cb6165c307a..e666f54d4c4 100644 --- a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.reference +++ b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.reference @@ -1,4 +1,3 @@ -1 0 --- 1 diff --git a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql index 045b7258a34..3a2e24d6bfe 100644 --- a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql +++ b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql @@ -3,13 +3,13 @@ SYSTEM DROP QUERY CACHE; --- rand() is non-deterministic, with default settings no entry in the query cache should be created -SELECT COUNT(rand(1)) SETTINGS use_query_cache = true; +-- rand() is non-deterministic, the query is rejected by default +SELECT COUNT(rand(1)) SETTINGS use_query_cache = true; -- { serverError 703 } SELECT COUNT(*) FROM system.query_cache; SELECT '---'; --- But an entry can be forced using a setting +-- Force caching using a setting SELECT COUNT(RAND(1)) SETTINGS use_query_cache = true, query_cache_store_results_of_queries_with_nondeterministic_functions = true; SELECT COUNT(*) FROM system.query_cache; From accbe24e08a92254c062f46d57e72a700334f10e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Sep 2023 18:42:46 +0200 Subject: [PATCH 79/93] Fix test --- tests/integration/test_check_table/test.py | 33 ++++++++-------------- 1 file changed, 12 insertions(+), 21 deletions(-) diff --git a/tests/integration/test_check_table/test.py b/tests/integration/test_check_table/test.py index 613ac3fb35f..99a5846d4ee 100644 --- a/tests/integration/test_check_table/test.py +++ b/tests/integration/test_check_table/test.py @@ -109,21 +109,15 @@ def test_check_normal_table_corruption(started_cluster): corrupt_data_part_on_disk(node1, "non_replicated_mt", "201902_1_1_0") - assert ( - node1.query( - "CHECK TABLE non_replicated_mt", - settings={"check_query_single_value_result": 0}, - ).strip() - == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 25." - ) + assert node1.query( + "CHECK TABLE non_replicated_mt", + settings={"check_query_single_value_result": 0}, + ).strip().split("\t")[0:2] == ["201902_1_1_0", "0"] - assert ( - node1.query( - "CHECK TABLE non_replicated_mt", - settings={"check_query_single_value_result": 0}, - ).strip() - == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 25." - ) + assert node1.query( + "CHECK TABLE non_replicated_mt", + settings={"check_query_single_value_result": 0}, + ).strip().split("\t")[0:2] == ["201902_1_1_0", "0"] node1.query( "INSERT INTO non_replicated_mt VALUES (toDate('2019-01-01'), 1, 10), (toDate('2019-01-01'), 2, 12)" @@ -141,13 +135,10 @@ def test_check_normal_table_corruption(started_cluster): remove_checksums_on_disk(node1, "non_replicated_mt", "201901_2_2_0") - assert ( - node1.query( - "CHECK TABLE non_replicated_mt PARTITION 201901", - settings={"check_query_single_value_result": 0}, - ) - == "201901_2_2_0\t0\tCheck of part finished with error: \\'Cannot read all data. Bytes read: 2. Bytes expected: 25.\\'\n" - ) + assert node1.query( + "CHECK TABLE non_replicated_mt PARTITION 201901", + settings={"check_query_single_value_result": 0}, + ).strip().split("\t")[0:2] == ["201901_2_2_0", "0"] def test_check_replicated_table_simple(started_cluster): From 5111f1e0901f13a758408a7aee39baba586eeb29 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 19 Sep 2023 18:59:00 +0200 Subject: [PATCH 80/93] Update docs/en/operations/query-cache.md Co-authored-by: Nikita Taranov --- docs/en/operations/query-cache.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index e111206355e..6e21b0b3658 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -158,7 +158,7 @@ Also, results of queries with non-deterministic functions are not cached by defa - functions which depend on the environment: [`currentUser()`](../sql-reference/functions/other-functions.md#currentUser), [`queryID()`](../sql-reference/functions/other-functions.md#queryID), [`getMacro()`](../sql-reference/functions/other-functions.md#getMacro) etc. -To force caching of results of queries with non-deterministic functionsregardless, using setting +To force caching of results of queries with non-deterministic functions regardless, use setting [query_cache_store_results_of_queries_with_nondeterministic_functions](settings/settings.md#query-cache-store-results-of-queries-with-nondeterministic-functions). Finally, entries in the query cache are not shared between users due to security reasons. For example, user A must not be able to bypass a From c439c4bca212a784c752f791b146f71de8fa883a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 19 Sep 2023 21:39:21 +0200 Subject: [PATCH 81/93] Revert "Fix filtering parts with indexHint for non analyzer" --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 5 +---- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 3 --- tests/analyzer_tech_debt.txt | 1 - .../0_stateless/02880_indexHint__partition_id.reference | 9 --------- .../0_stateless/02880_indexHint__partition_id.sql | 9 --------- 5 files changed, 1 insertion(+), 26 deletions(-) delete mode 100644 tests/queries/0_stateless/02880_indexHint__partition_id.reference delete mode 100644 tests/queries/0_stateless/02880_indexHint__partition_id.sql diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 237a4cc703f..a2f2c1e0aac 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1346,10 +1346,7 @@ static void buildIndexes( } /// TODO Support row_policy_filter and additional_filters - if (settings.allow_experimental_analyzer) - indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context); - else - indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, query_info.query, context); + indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context); indexes->use_skip_indexes = settings.use_skip_indexes; bool final = query_info.isFinal(); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index d7c6c370c18..3c2b09b6f3b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -815,9 +815,6 @@ std::optional> MergeTreeDataSelectExecutor::filterPar ASTPtr expression_ast; auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */); - if (virtual_columns_block.rows() == 0) - return {}; - // Generate valid expressions for filtering VirtualColumnUtils::prepareFilterBlockWithQuery(query, context, virtual_columns_block, expression_ast); diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 652ab0b99de..4419190e12c 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -54,7 +54,6 @@ 01710_projection_additional_filters 01721_join_implicit_cast_long 01739_index_hint -02880_indexHint__partition_id 01747_join_view_filter_dictionary 01748_partition_id_pruning 01756_optimize_skip_unused_shards_rewrite_in diff --git a/tests/queries/0_stateless/02880_indexHint__partition_id.reference b/tests/queries/0_stateless/02880_indexHint__partition_id.reference deleted file mode 100644 index 365e7b676c7..00000000000 --- a/tests/queries/0_stateless/02880_indexHint__partition_id.reference +++ /dev/null @@ -1,9 +0,0 @@ --- { echoOn } -select * from data prewhere indexHint(_partition_id = '1'); -1 -select count() from data prewhere indexHint(_partition_id = '1'); -1 -select * from data where indexHint(_partition_id = '1'); -1 -select count() from data where indexHint(_partition_id = '1'); -1 diff --git a/tests/queries/0_stateless/02880_indexHint__partition_id.sql b/tests/queries/0_stateless/02880_indexHint__partition_id.sql deleted file mode 100644 index d15b3f4ccea..00000000000 --- a/tests/queries/0_stateless/02880_indexHint__partition_id.sql +++ /dev/null @@ -1,9 +0,0 @@ -drop table if exists data; -create table data (part Int) engine=MergeTree() order by tuple() partition by part; -insert into data values (1)(2); - --- { echoOn } -select * from data prewhere indexHint(_partition_id = '1'); -select count() from data prewhere indexHint(_partition_id = '1'); -select * from data where indexHint(_partition_id = '1'); -select count() from data where indexHint(_partition_id = '1'); From fa51399ede28f551a62061d6961032d1c99e763a Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 19 Sep 2023 21:39:59 +0000 Subject: [PATCH 82/93] Make clickhouse-local logging (server_logs_file) prepend timestamps etc --- programs/local/LocalServer.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index d56e7d8b47c..9fb629a0871 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -32,6 +32,8 @@ #include #include #include +#include +#include #include #include #include @@ -599,7 +601,9 @@ void LocalServer::processConfig() { auto poco_logs_level = Poco::Logger::parseLevel(level); Poco::Logger::root().setLevel(poco_logs_level); - Poco::Logger::root().setChannel(Poco::AutoPtr(new Poco::SimpleFileChannel(server_logs_file))); + Poco::AutoPtr pf = new OwnPatternFormatter; + Poco::AutoPtr log = new OwnFormattingChannel(pf, new Poco::SimpleFileChannel(server_logs_file)); + Poco::Logger::root().setChannel(log); logging_initialized = true; } else if (logging || is_interactive) From 7271cfd18704e2ab89ec9d8f3528e67794b6c7b0 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 19 Sep 2023 21:41:02 +0000 Subject: [PATCH 83/93] Prevent parquet schema inference reading the first 1 MB of the file unnecessarily --- src/Formats/ReadSchemaUtils.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 3158798fdca..b185007eda7 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -86,7 +87,16 @@ try buf = read_buffer_iterator.next(); if (!buf) break; - is_eof = buf->eof(); + + /// We just want to check for eof, but eof() can be pretty expensive. + /// So we use getFileSize() when available, which has better worst case. + /// (For remote files, typically eof() would read 1 MB from S3, which may be much + /// more than what the schema reader and even data reader will read). + auto size = tryGetFileSizeFromReadBuffer(*buf); + if (size.has_value()) + is_eof = *size == 0; + else + is_eof = buf->eof(); } catch (Exception & e) { From c856ec4087bb990c6fe1c22d6b619a6ae8646e31 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 19 Sep 2023 21:43:44 +0000 Subject: [PATCH 84/93] Prevent ParquetMetadata reading 40 MB from each file unnecessarily --- .../Formats/Impl/ParquetMetadataInputFormat.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp index 043e6d2260c..1f81f5ac201 100644 --- a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp @@ -130,7 +130,7 @@ static std::shared_ptr getFileMetadata( const FormatSettings & format_settings, std::atomic & is_stopped) { - auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES); + auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); return parquet::ReadMetaData(arrow_file); } @@ -495,12 +495,15 @@ NamesAndTypesList ParquetMetadataSchemaReader::readSchema() void registerInputFormatParquetMetadata(FormatFactory & factory) { - factory.registerInputFormat( + factory.registerRandomAccessInputFormat( "ParquetMetadata", - [](ReadBuffer &buf, - const Block &sample, - const RowInputFormatParams &, - const FormatSettings & settings) + [](ReadBuffer & buf, + const Block & sample, + const FormatSettings & settings, + const ReadSettings &, + bool /* is_remote_fs */, + size_t /* max_download_threads */, + size_t /* max_parsing_threads */) { return std::make_shared(buf, sample, settings); }); From e8cd42945388ecc20192b398da17867955873160 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 19 Sep 2023 22:11:58 +0000 Subject: [PATCH 85/93] add error code INVALID_IDENTIFIER --- src/Common/ErrorCodes.cpp | 1 + src/Interpreters/DatabaseAndTableWithAlias.cpp | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f23685c37d1..e2ad8741ca3 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -585,6 +585,7 @@ M(700, USER_SESSION_LIMIT_EXCEEDED) \ M(701, CLUSTER_DOESNT_EXIST) \ M(702, CLIENT_INFO_DOES_NOT_MATCH) \ + M(703, INVALID_IDENTIFIER) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Interpreters/DatabaseAndTableWithAlias.cpp b/src/Interpreters/DatabaseAndTableWithAlias.cpp index 9b6ce4f22d3..cb993611ecb 100644 --- a/src/Interpreters/DatabaseAndTableWithAlias.cpp +++ b/src/Interpreters/DatabaseAndTableWithAlias.cpp @@ -16,6 +16,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int INVALID_IDENTIFIER; } DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableIdentifier & identifier, const String & current_database) @@ -37,7 +38,7 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & ident else if (identifier.name_parts.size() == 1) table = identifier.name_parts[0]; else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: invalid identifier"); + throw Exception(ErrorCodes::INVALID_IDENTIFIER, "Invalid identifier"); if (database.empty()) database = current_database; @@ -50,7 +51,7 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTPtr & node, const else if (const auto * identifier = node->as()) *this = DatabaseAndTableWithAlias(*identifier, current_database); else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: identifier or table identifier expected"); + throw Exception(ErrorCodes::INVALID_IDENTIFIER, "Identifier or table identifier expected"); } DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database) From e9e5c7a09bdfc36459d7a2ac141914a9357c67f7 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 20 Sep 2023 08:44:30 +0800 Subject: [PATCH 86/93] update --- src/Functions/FunctionSQLJSON.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index 1786b613f98..0533f3d419a 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -209,7 +209,7 @@ public: bool added_to_column = false; if (document_ok) { - // Instead of creating a new generator for each row, we can reuse the same one. + /// Instead of creating a new generator for each row, we can reuse the same one. generator_json_path.reinitialize(); added_to_column = impl.insertResultToColumn(*to, document, generator_json_path, context); } From 56dca144cc4a35c953113ab0aa7f0bf3448cbf73 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 20 Sep 2023 07:09:31 +0200 Subject: [PATCH 87/93] Fix test_backup_restore_on_cluster/test.py::test_stop_other_host_during_backup flakiness Signed-off-by: Azat Khuzhin --- tests/integration/test_backup_restore_on_cluster/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index dfce2f15413..20f538cca58 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -1096,6 +1096,7 @@ def test_stop_other_host_during_backup(kill): if status == "BACKUP_CREATED": node1.query("DROP TABLE tbl ON CLUSTER 'cluster' SYNC") node1.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") + node1.query("SYSTEM SYNC REPLICA tbl") assert node1.query("SELECT * FROM tbl ORDER BY x") == TSV([3, 5]) elif status == "BACKUP_FAILED": assert not os.path.exists( From 696ab745aa44da168f98d7c024f325fc08c3be34 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 20 Sep 2023 07:51:27 +0200 Subject: [PATCH 88/93] Doc. Update query-complexity.md --- docs/en/operations/settings/query-complexity.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index b65ecdcb6ab..15f39b53e07 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -157,7 +157,7 @@ At this time, it is not checked for one of the sorting stages, or when merging a The `max_execution_time` parameter can be a bit tricky to understand. It operates based on interpolation relative to the current query execution speed (this behaviour is controlled by [timeout_before_checking_execution_speed](#timeout-before-checking-execution-speed)). ClickHouse will interrupt a query if the projected execution time exceeds the specified `max_execution_time`. -By default, the timeout_before_checking_execution_speed is set to 1 second. This means that after just one second of query execution, ClickHouse will begin estimating the total execution time. +By default, the timeout_before_checking_execution_speed is set to 10 seconds. This means that after 10 seconds of query execution, ClickHouse will begin estimating the total execution time. If, for example, `max_execution_time` is set to 3600 seconds (1 hour), ClickHouse will terminate the query if the estimated time exceeds this 3600-second limit. If you set `timeout_before_checking_execution_speed `to 0, ClickHouse will use clock time as the basis for `max_execution_time`. From a175a7e0fcd696e08f3262582b5dbb7074259668 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 20 Sep 2023 09:04:35 +0000 Subject: [PATCH 89/93] Use error name instead of error code --- .../02494_query_cache_nondeterministic_functions.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql index 3a2e24d6bfe..62e0b099d7a 100644 --- a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql +++ b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql @@ -4,7 +4,7 @@ SYSTEM DROP QUERY CACHE; -- rand() is non-deterministic, the query is rejected by default -SELECT COUNT(rand(1)) SETTINGS use_query_cache = true; -- { serverError 703 } +SELECT COUNT(rand(1)) SETTINGS use_query_cache = true; -- { serverError CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS } SELECT COUNT(*) FROM system.query_cache; SELECT '---'; From 65baf129bf8a081d4041fa73aebb34287fcb098b Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 19 Sep 2023 15:34:42 +0000 Subject: [PATCH 90/93] Do not set PR status label --- tests/ci/commit_status_helper.py | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index a5fd27efb6b..3d07d81a1fe 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -141,16 +141,6 @@ STATUS_ICON_MAP = defaultdict( ) -def update_pr_status_label(pr: PullRequest, status: str) -> None: - new_label = "pr-status-" + STATUS_ICON_MAP[status] - for label in pr.get_labels(): - if label.name == new_label: - return - if label.name.startswith("pr-status-"): - pr.remove_from_labels(label.name) - pr.add_to_labels(new_label) - - def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: """It adds or updates the comment status to all Pull Requests but for release one, so the method does nothing for simple pushes and pull requests with @@ -190,8 +180,6 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: comment = ic break - update_pr_status_label(pr, get_worst_state(statuses)) - if comment is None: pr.create_issue_comment(comment_body) return From d8adf05de25a61555ded0cdbec3dc2388d77909e Mon Sep 17 00:00:00 2001 From: SmitaRKulkarni <64093672+SmitaRKulkarni@users.noreply.github.com> Date: Wed, 20 Sep 2023 11:31:12 +0200 Subject: [PATCH 91/93] Added a new column _block_number (#47532) Added a new virtual column _block_number which is persisted on merges when allow_experimental_block_number_column is enabled --- .../mergetree-family/mergetree.md | 1 + .../settings/merge-tree-settings.md | 6 +++ src/Interpreters/InterpreterCreateQuery.cpp | 8 ++++ src/Interpreters/MutationsInterpreter.cpp | 26 +++++++++++- src/Interpreters/inplaceBlockConversions.cpp | 14 +++++-- src/Interpreters/inplaceBlockConversions.h | 2 +- .../Algorithms/SummingSortedAlgorithm.cpp | 7 ++++ src/Processors/Transforms/TTLTransform.cpp | 3 +- src/Storages/AlterCommands.cpp | 11 ++++- src/Storages/BlockNumberColumn.cpp | 23 +++++++++++ src/Storages/BlockNumberColumn.h | 16 ++++++++ src/Storages/ColumnsDescription.cpp | 6 +++ src/Storages/MergeTree/IMergeTreeDataPart.h | 4 -- src/Storages/MergeTree/IMergeTreeReader.cpp | 4 +- src/Storages/MergeTree/IMergeTreeReader.h | 2 +- src/Storages/MergeTree/MergeTask.cpp | 23 ++++++++++- src/Storages/MergeTree/MergeTask.h | 7 ++++ src/Storages/MergeTree/MergeTreeData.cpp | 4 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 7 ++++ .../MergeTreeDataPartWriterCompact.cpp | 12 +++++- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 12 +++++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 ++- .../MergeTree/MergeTreeSelectProcessor.cpp | 27 ++++++++++-- .../MergeTree/MergeTreeSequentialSource.cpp | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 3 +- src/Storages/StorageDistributed.cpp | 3 ++ src/Storages/StorageLog.cpp | 12 +++++- src/Storages/StorageSnapshot.cpp | 2 + .../02668_column_block_number.reference | 41 +++++++++++++++++++ .../0_stateless/02668_column_block_number.sql | 32 +++++++++++++++ ...lumn_block_number_vertical_merge.reference | 41 +++++++++++++++++++ ...668_column_block_number_vertical_merge.sql | 36 ++++++++++++++++ ...mn_block_number_with_projections.reference | 19 +++++++++ ...8_column_block_number_with_projections.sql | 18 ++++++++ 34 files changed, 411 insertions(+), 29 deletions(-) create mode 100644 src/Storages/BlockNumberColumn.cpp create mode 100644 src/Storages/BlockNumberColumn.h create mode 100644 tests/queries/0_stateless/02668_column_block_number.reference create mode 100644 tests/queries/0_stateless/02668_column_block_number.sql create mode 100644 tests/queries/0_stateless/02668_column_block_number_vertical_merge.reference create mode 100644 tests/queries/0_stateless/02668_column_block_number_vertical_merge.sql create mode 100644 tests/queries/0_stateless/02668_column_block_number_with_projections.reference create mode 100644 tests/queries/0_stateless/02668_column_block_number_with_projections.sql diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index bb257311680..23ab2699cc1 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1354,3 +1354,4 @@ In this sample configuration: - `_part_uuid` — Unique part identifier (if enabled MergeTree setting `assign_part_uuids`). - `_partition_value` — Values (a tuple) of a `partition by` expression. - `_sample_factor` — Sample factor (from the query). +- `_block_number` — Block number of the row, it is persisted on merges when `allow_experimental_block_number_column` is set to true. diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index e746719b629..483dcf2e61c 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -854,3 +854,9 @@ Possible values: - `Always` or `Never`. Default value: `Never` + +## allow_experimental_block_number_column + +Persists virtual column `_block_number` on merges. + +Default value: false. diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index b426025413c..3654f307eb9 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -31,6 +31,7 @@ #include #include #include +#include #include #include @@ -837,6 +838,13 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat "Cannot create table with column '{}' for *MergeTree engines because it " "is reserved for lightweight delete feature", LightweightDeleteDescription::FILTER_COLUMN.name); + + auto search_block_number = all_columns.find(BlockNumberColumn::name); + if (search_block_number != all_columns.end()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Cannot create table with column '{}' for *MergeTree engines because it " + "is reserved for storing block number", + BlockNumberColumn::name); } const auto & settings = getContext()->getSettingsRef(); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index e50f8488cac..4b0cbec4f9f 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -40,7 +41,6 @@ #include #include - namespace DB { @@ -56,6 +56,7 @@ namespace ErrorCodes extern const int THERE_IS_NO_COLUMN; } + namespace { @@ -416,6 +417,12 @@ static void validateUpdateColumns( found = true; } + /// Dont allow to override value of block number virtual column + if (!found && column_name == BlockNumberColumn::name) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Update is not supported for virtual column {} ", backQuote(column_name)); + } + if (!found) { for (const auto & col : metadata_snapshot->getColumns().getMaterialized()) @@ -511,7 +518,8 @@ void MutationsInterpreter::prepare(bool dry_run) for (const auto & [name, _] : command.column_to_update_expression) { - if (!available_columns_set.contains(name) && name != LightweightDeleteDescription::FILTER_COLUMN.name) + if (!available_columns_set.contains(name) && name != LightweightDeleteDescription::FILTER_COLUMN.name + && name != BlockNumberColumn::name) throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Column {} is updated but not requested to read", name); @@ -613,6 +621,8 @@ void MutationsInterpreter::prepare(bool dry_run) type = physical_column->type; else if (column == LightweightDeleteDescription::FILTER_COLUMN.name) type = LightweightDeleteDescription::FILTER_COLUMN.type; + else if (column == BlockNumberColumn::name) + type = BlockNumberColumn::type; else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown column {}", column); @@ -1087,6 +1097,18 @@ struct VirtualColumns virtuals.emplace_back(ColumnAndPosition{.column = std::move(column), .position = i}); } + else if (columns_to_read[i] == BlockNumberColumn::name) + { + if (!part->getColumns().contains(BlockNumberColumn::name)) + { + ColumnWithTypeAndName block_number_column; + block_number_column.type = BlockNumberColumn::type; + block_number_column.column = block_number_column.type->createColumnConst(0, part->info.min_block); + block_number_column.name = std::move(columns_to_read[i]); + + virtuals.emplace_back(ColumnAndPosition{.column = std::move(block_number_column), .position = i}); + } + } } if (!virtuals.empty()) diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 4cac2f0e20c..b7ef5dbdbbc 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -20,6 +20,7 @@ #include #include #include +#include namespace DB @@ -260,7 +261,7 @@ void fillMissingColumns( const NamesAndTypesList & requested_columns, const NamesAndTypesList & available_columns, const NameSet & partially_read_columns, - StorageMetadataPtr metadata_snapshot) + StorageMetadataPtr metadata_snapshot, size_t block_number) { size_t num_columns = requested_columns.size(); if (num_columns != res_columns.size()) @@ -339,9 +340,14 @@ void fillMissingColumns( } else { - /// We must turn a constant column into a full column because the interpreter could infer - /// that it is constant everywhere but in some blocks (from other parts) it can be a full column. - res_columns[i] = type->createColumnConstWithDefaultValue(num_rows)->convertToFullColumnIfConst(); + if (requested_column->name == BlockNumberColumn::name) + res_columns[i] = type->createColumnConst(num_rows, block_number)->convertToFullColumnIfConst(); + else + /// We must turn a constant column into a full column because the interpreter could infer + /// that it is constant everywhere but in some blocks (from other parts) it can be a full column. + res_columns[i] = type->createColumnConstWithDefaultValue(num_rows)->convertToFullColumnIfConst(); + + } } } diff --git a/src/Interpreters/inplaceBlockConversions.h b/src/Interpreters/inplaceBlockConversions.h index bea44bf6db9..7a13a75ec8b 100644 --- a/src/Interpreters/inplaceBlockConversions.h +++ b/src/Interpreters/inplaceBlockConversions.h @@ -46,6 +46,6 @@ void fillMissingColumns( const NamesAndTypesList & requested_columns, const NamesAndTypesList & available_columns, const NameSet & partially_read_columns, - StorageMetadataPtr metadata_snapshot); + StorageMetadataPtr metadata_snapshot, size_t block_number = 0); } diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index 7dac5715f95..f468e112d8f 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB @@ -222,6 +223,12 @@ static SummingSortedAlgorithm::ColumnsDefinition defineColumns( const ColumnWithTypeAndName & column = header.safeGetByPosition(i); const auto * simple = dynamic_cast(column.type->getCustomName()); + if (column.name == BlockNumberColumn::name) + { + def.column_numbers_not_to_aggregate.push_back(i); + continue; + } + /// Discover nested Maps and find columns for summation if (typeid_cast(column.type.get()) && !simple) { diff --git a/src/Processors/Transforms/TTLTransform.cpp b/src/Processors/Transforms/TTLTransform.cpp index 3250d012d5c..7cde86098c7 100644 --- a/src/Processors/Transforms/TTLTransform.cpp +++ b/src/Processors/Transforms/TTLTransform.cpp @@ -49,7 +49,8 @@ TTLTransform::TTLTransform( for (const auto & group_by_ttl : metadata_snapshot_->getGroupByTTLs()) algorithms.emplace_back(std::make_unique( - group_by_ttl, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_, getInputPort().getHeader(), storage_)); + group_by_ttl, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_, + getInputPort().getHeader(), storage_)); if (metadata_snapshot_->hasAnyColumnTTL()) { diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index f38fc1f3734..da46cb4d7fe 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -782,7 +783,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada /// Drop alias is metadata alter, in other case mutation is required. if (type == DROP_COLUMN) return metadata.columns.hasColumnOrNested(GetColumnsOptions::AllPhysical, column_name) || - column_name == LightweightDeleteDescription::FILTER_COLUMN.name; + column_name == LightweightDeleteDescription::FILTER_COLUMN.name || column_name == BlockNumberColumn::name; if (type != MODIFY_COLUMN || data_type == nullptr) return false; @@ -1066,6 +1067,10 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add column {}: " "this column name is reserved for lightweight delete feature", backQuote(column_name)); + if (column_name == BlockNumberColumn::name && std::dynamic_pointer_cast(table)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add column {}: " + "this column name is reserved for _block_number persisting feature", backQuote(column_name)); + if (command.codec) CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_deflate_qpl_codec); @@ -1270,6 +1275,10 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot rename to {}: " "this column name is reserved for lightweight delete feature", backQuote(command.rename_to)); + if (command.rename_to == BlockNumberColumn::name && std::dynamic_pointer_cast(table)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot rename to {}: " + "this column name is reserved for _block_number persisting feature", backQuote(command.rename_to)); + if (modified_columns.contains(column_name)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot rename and modify the same column {} " "in a single ALTER query", backQuote(column_name)); diff --git a/src/Storages/BlockNumberColumn.cpp b/src/Storages/BlockNumberColumn.cpp new file mode 100644 index 00000000000..8c9e1fd902a --- /dev/null +++ b/src/Storages/BlockNumberColumn.cpp @@ -0,0 +1,23 @@ +#include +#include + +namespace DB +{ + +CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); + +CompressionCodecPtr getCompressionCodecForBlockNumberColumn() +{ + std::vector codecs; + codecs.reserve(2); + auto data_bytes_size = BlockNumberColumn::type->getSizeOfValueInMemory(); + codecs.emplace_back(getCompressionCodecDelta(data_bytes_size)); + codecs.emplace_back(CompressionCodecFactory::instance().get("LZ4", {})); + return std::make_shared(codecs); +} + +const String BlockNumberColumn::name = "_block_number"; +const DataTypePtr BlockNumberColumn::type = std::make_shared(); +const CompressionCodecPtr BlockNumberColumn::compression_codec = getCompressionCodecForBlockNumberColumn(); + +} diff --git a/src/Storages/BlockNumberColumn.h b/src/Storages/BlockNumberColumn.h new file mode 100644 index 00000000000..fffa68bfd49 --- /dev/null +++ b/src/Storages/BlockNumberColumn.h @@ -0,0 +1,16 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +struct BlockNumberColumn +{ + static const String name; + static const DataTypePtr type; + static const CompressionCodecPtr compression_codec; +}; + +} diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 0c918bda5fd..9fa79387d5c 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -30,11 +30,15 @@ #include #include #include +#include namespace DB { +CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); + + namespace ErrorCodes { extern const int NO_SUCH_COLUMN_IN_TABLE; @@ -721,11 +725,13 @@ CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_ CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_name) const { + assert (column_name != BlockNumberColumn::name); return getCodecOrDefault(column_name, CompressionCodecFactory::instance().getDefaultCodec()); } ASTPtr ColumnsDescription::getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const { + assert (column_name != BlockNumberColumn::name); const auto it = columns.get<1>().find(column_name); if (it == columns.get<1>().end() || !it->codec) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 97c9b81ce87..02c838458f9 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -477,10 +477,6 @@ public: /// Moar hardening: this method is supposed to be used for debug assertions bool assertHasValidVersionMetadata() const; - /// Return hardlink count for part. - /// Required for keep data on remote FS when part has shadow copies. - UInt32 getNumberOfRefereneces() const; - /// True if the part supports lightweight delete mutate. bool supportLightweightDeleteMutate() const; diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 4bb8c400691..120edd81e30 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -62,7 +62,7 @@ const IMergeTreeReader::ValueSizeMap & IMergeTreeReader::getAvgValueSizeHints() return avg_value_size_hints; } -void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows) const +void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows, size_t block_number) const { try { @@ -71,7 +71,7 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e res_columns, num_rows, Nested::convertToSubcolumns(requested_columns), Nested::convertToSubcolumns(available_columns), - partially_read_columns, storage_snapshot->metadata); + partially_read_columns, storage_snapshot->metadata, block_number); should_evaluate_missing_defaults = std::any_of( res_columns.begin(), res_columns.end(), [](const auto & column) { return column == nullptr; }); diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index a7e60254217..02faebf4b41 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -45,7 +45,7 @@ public: /// Add columns from ordered_names that are not present in the block. /// Missing columns are added in the order specified by ordered_names. /// num_rows is needed in case if all res_columns are nullptr. - void fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows) const; + void fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows, size_t block_number = 0) const; /// Evaluate defaulted columns if necessary. void evaluateMissingDefaults(Block additional_columns, Columns & res_columns) const; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index df607d36402..adb1ca72e46 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -218,6 +218,14 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() ctx->need_remove_expired_values = false; ctx->force_ttl = false; + if (supportsBlockNumberColumn(global_ctx) && !global_ctx->storage_columns.contains(BlockNumberColumn::name)) + { + global_ctx->storage_columns.emplace_back(NameAndTypePair{BlockNumberColumn::name,BlockNumberColumn::type}); + global_ctx->all_column_names.emplace_back(BlockNumberColumn::name); + global_ctx->gathering_columns.emplace_back(NameAndTypePair{BlockNumberColumn::name,BlockNumberColumn::type}); + global_ctx->gathering_column_names.emplace_back(BlockNumberColumn::name); + } + SerializationInfo::Settings info_settings = { .ratio_of_defaults_for_sparse = global_ctx->data->getSettings()->ratio_of_defaults_for_sparse_serialization, @@ -251,12 +259,12 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() } } - global_ctx->new_data_part->setColumns(global_ctx->storage_columns, infos, global_ctx->metadata_snapshot->getMetadataVersion()); - const auto & local_part_min_ttl = global_ctx->new_data_part->ttl_infos.part_min_ttl; if (local_part_min_ttl && local_part_min_ttl <= global_ctx->time_of_merge) ctx->need_remove_expired_values = true; + global_ctx->new_data_part->setColumns(global_ctx->storage_columns, infos, global_ctx->metadata_snapshot->getMetadataVersion()); + if (ctx->need_remove_expired_values && global_ctx->ttl_merges_blocker->isCancelled()) { LOG_INFO(ctx->log, "Part {} has values with expired TTL, but merges with TTL are cancelled.", global_ctx->new_data_part->name); @@ -998,6 +1006,17 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() if (global_ctx->deduplicate) { + /// We don't want to deduplicate by block number column + /// so if deduplicate_by_columns is empty, add all columns except _block_number + if (supportsBlockNumberColumn(global_ctx) && global_ctx->deduplicate_by_columns.empty()) + { + for (const auto & col : global_ctx->merging_column_names) + { + if (col != BlockNumberColumn::name) + global_ctx->deduplicate_by_columns.emplace_back(col); + } + } + if (DistinctSortedTransform::isApplicable(header, sort_description, global_ctx->deduplicate_by_columns)) res_pipe.addTransform(std::make_shared( res_pipe.getHeader(), sort_description, SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns)); diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 402d3c26e49..2122484bb24 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -388,6 +389,12 @@ private: Stages::iterator stages_iterator = stages.begin(); + /// Check for persisting block number column + static bool supportsBlockNumberColumn(GlobalRuntimeContextPtr global_ctx) + { + return global_ctx->data->getSettings()->allow_experimental_block_number_column && global_ctx->metadata_snapshot->getGroupByTTLs().empty(); + } + }; /// FIXME diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4b6d2ea41ed..14c9961f6c3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -78,6 +78,7 @@ #include #include #include +#include #include #include @@ -3730,7 +3731,7 @@ void MergeTreeData::checkPartDynamicColumns(MutableDataPartPtr & part, DataParts const auto & part_columns = part->getColumns(); for (const auto & part_column : part_columns) { - if (part_column.name == LightweightDeleteDescription::FILTER_COLUMN.name) + if (part_column.name == LightweightDeleteDescription::FILTER_COLUMN.name || part_column.name == BlockNumberColumn::name) continue; auto storage_column = columns.getPhysical(part_column.name); @@ -8269,6 +8270,7 @@ NamesAndTypesList MergeTreeData::getVirtuals() const NameAndTypePair("_sample_factor", std::make_shared()), NameAndTypePair("_part_offset", std::make_shared()), LightweightDeleteDescription::FILTER_COLUMN, + NameAndTypePair(BlockNumberColumn::name, BlockNumberColumn::type), }; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index c6d059498ff..209a5061f33 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -64,6 +65,12 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( ordered_columns_list.sort([this](const auto & lhs, const auto & rhs) { return *getColumnPosition(lhs.name) < *getColumnPosition(rhs.name); }); + /// _block_number column is not added by user, but is persisted in a part after merge + /// If _block_number is not present in the parts to be merged, then it won't have a position + /// So check if its not present and add it at the end + if (columns_list.contains(BlockNumberColumn::name) && !ordered_columns_list.contains(BlockNumberColumn::name)) + ordered_columns_list.emplace_back(NameAndTypePair{BlockNumberColumn::name, BlockNumberColumn::type}); + return std::make_unique( shared_from_this(), ordered_columns_list, metadata_snapshot, indices_to_recalc, getMarksFileExtension(), diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 3475130bf24..d2a9632d4e5 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -1,9 +1,12 @@ #include #include +#include namespace DB { + CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -53,7 +56,14 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const auto & storage_columns = metadata_snapshot->getColumns(); for (const auto & column : columns_list) - addStreams(column, storage_columns.getCodecDescOrDefault(column.name, default_codec)); + { + ASTPtr compression; + if (column.name == BlockNumberColumn::name) + compression = BlockNumberColumn::compression_codec->getFullCodecDesc(); + else + compression = storage_columns.getCodecDescOrDefault(column.name, default_codec); + addStreams(column, compression); + } } void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, const ASTPtr & effective_codec_desc) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index bcf340e0f55..f3e60e224aa 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -6,9 +6,12 @@ #include #include #include +#include namespace DB { + CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -87,7 +90,14 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( { const auto & columns = metadata_snapshot->getColumns(); for (const auto & it : columns_list) - addStreams(it, columns.getCodecDescOrDefault(it.name, default_codec)); + { + ASTPtr compression; + if (it.name == BlockNumberColumn::name) + compression = BlockNumberColumn::compression_codec->getFullCodecDesc(); + else + compression = columns.getCodecDescOrDefault(it.name, default_codec); + addStreams(it, compression); + } } void MergeTreeDataPartWriterWide::addStreams( diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 3c2b09b6f3b..d8ebfb95d74 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -46,7 +46,7 @@ #include #include - +#include #include namespace CurrentMetrics @@ -1232,6 +1232,10 @@ static void selectColumnNames( { virt_column_names.push_back(name); } + else if (name == BlockNumberColumn::name) + { + virt_column_names.push_back(name); + } else if (name == "_part_uuid") { virt_column_names.push_back(name); diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 95fcde23f8e..9b480ac27a0 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include namespace DB @@ -24,7 +25,8 @@ namespace ErrorCodes static void injectNonConstVirtualColumns( size_t rows, Block & block, - const Names & virtual_columns); + const Names & virtual_columns, + MergeTreeReadTask * task = nullptr); static void injectPartConstVirtualColumns( size_t rows, @@ -247,7 +249,8 @@ namespace static void injectNonConstVirtualColumns( size_t rows, Block & block, - const Names & virtual_columns) + const Names & virtual_columns, + MergeTreeReadTask * task) { VirtualColumnsInserter inserter(block); for (const auto & virtual_column_name : virtual_columns) @@ -278,6 +281,24 @@ static void injectNonConstVirtualColumns( inserter.insertUInt8Column(column, virtual_column_name); } + + if (virtual_column_name == BlockNumberColumn::name) + { + ColumnPtr column; + if (rows) + { + size_t value = 0; + if (task) + { + value = task->getInfo().data_part ? task->getInfo().data_part->info.min_block : 0; + } + column = BlockNumberColumn::type->createColumnConst(rows, value)->convertToFullColumnIfConst(); + } + else + column = BlockNumberColumn::type->createColumn(); + + inserter.insertUInt64Column(column, virtual_column_name); + } } } @@ -368,7 +389,7 @@ void MergeTreeSelectProcessor::injectVirtualColumns( { /// First add non-const columns that are filled by the range reader and then const columns that we will fill ourselves. /// Note that the order is important: virtual columns filled by the range reader must go first - injectNonConstVirtualColumns(row_count, block, virtual_columns); + injectNonConstVirtualColumns(row_count, block, virtual_columns,task); injectPartConstVirtualColumns(row_count, block, task, partition_value_type, virtual_columns); } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 8a9faa5cee4..6ba0d4dcfad 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -176,7 +176,7 @@ try current_mark += (rows_to_read == rows_read); bool should_evaluate_missing_defaults = false; - reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_read); + reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_read, data_part->info.min_block); if (should_evaluate_missing_defaults) { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 4f36da048c2..52c3f3efc6d 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -171,7 +171,8 @@ struct Settings; M(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ M(Bool, allow_remote_fs_zero_copy_replication, false, "Don't use this setting in production, because it is not ready.", 0) \ M(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for zero-copy table-independent info.", 0) \ - M(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", 0) \ + M(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", 0) \ + M(Bool, allow_experimental_block_number_column, false, "Enable persisting column _block_number for each row.", 0) \ \ /** Compress marks and primary key. */ \ M(Bool, compress_marks, true, "Marks support compression, reduce mark file size and speed up network transmission.", 0) \ diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f869a1158ef..4baa43311d4 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -102,6 +102,8 @@ #include #include +#include + #include #include #include @@ -298,6 +300,7 @@ NamesAndTypesList StorageDistributed::getVirtuals() const NameAndTypePair("_sample_factor", std::make_shared()), NameAndTypePair("_part_offset", std::make_shared()), NameAndTypePair("_row_exists", std::make_shared()), + NameAndTypePair(BlockNumberColumn::name, BlockNumberColumn::type), NameAndTypePair("_shard_num", std::make_shared()), /// deprecated }; } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index ab795a2d04c..cf76f7a16ba 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include #include @@ -45,6 +46,8 @@ namespace DB { + CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); + namespace ErrorCodes { extern const int TIMEOUT_EXCEEDED; @@ -452,10 +455,15 @@ void LogSink::writeData(const NameAndTypePair & name_and_type, const IColumn & c const auto & data_file = *data_file_it->second; const auto & columns = metadata_snapshot->getColumns(); + CompressionCodecPtr compression; + if (name_and_type.name == BlockNumberColumn::name) + compression = BlockNumberColumn::compression_codec; + else + compression = columns.getCodecOrDefault(name_and_type.name); + it = streams.try_emplace(data_file.name, storage.disk, data_file.path, storage.file_checker.getFileSize(data_file.path), - columns.getCodecOrDefault(name_and_type.name), - storage.max_compress_block_size).first; + compression, storage.max_compress_block_size).first; } auto & stream = it->second; diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index c0e85900794..0c19634f50c 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -24,6 +25,7 @@ void StorageSnapshot::init() if (storage.hasLightweightDeletedMask()) system_columns[LightweightDeleteDescription::FILTER_COLUMN.name] = LightweightDeleteDescription::FILTER_COLUMN.type; + system_columns[BlockNumberColumn::name] = BlockNumberColumn::type; } NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options) const diff --git a/tests/queries/0_stateless/02668_column_block_number.reference b/tests/queries/0_stateless/02668_column_block_number.reference new file mode 100644 index 00000000000..f08220243ff --- /dev/null +++ b/tests/queries/0_stateless/02668_column_block_number.reference @@ -0,0 +1,41 @@ +*** BEFORE MUTATION BEFORE MERGE *** +1 1 1 all_1_1_0 +2 2 1 all_1_1_0 +3 3 1 all_1_1_0 +4 4 2 all_2_2_0 +5 5 2 all_2_2_0 +6 6 2 all_2_2_0 +*** AFTER MUTATION BEFORE MERGE *** +1 0 1 all_1_1_0_3 +2 0 1 all_1_1_0_3 +3 0 1 all_1_1_0_3 +4 4 2 all_2_2_0_3 +5 5 2 all_2_2_0_3 +6 6 2 all_2_2_0_3 +*** AFTER MUTATION AFTER MERGE *** +1 0 1 all_1_2_1_3 +2 0 1 all_1_2_1_3 +3 0 1 all_1_2_1_3 +4 4 2 all_1_2_1_3 +5 5 2 all_1_2_1_3 +6 6 2 all_1_2_1_3 +*** AFTER MUTATION AFTER MERGE , NEW BLOCK *** +1 0 1 all_1_2_1_3 +2 0 1 all_1_2_1_3 +3 0 1 all_1_2_1_3 +4 4 2 all_1_2_1_3 +5 5 2 all_1_2_1_3 +6 6 2 all_1_2_1_3 +7 7 4 all_4_4_0 +8 8 4 all_4_4_0 +9 9 4 all_4_4_0 +*** AFTER MUTATION AFTER MERGE , NEW BLOCK MERGED *** +1 0 1 all_1_4_2_3 +2 0 1 all_1_4_2_3 +3 0 1 all_1_4_2_3 +4 4 2 all_1_4_2_3 +5 5 2 all_1_4_2_3 +6 6 2 all_1_4_2_3 +7 7 4 all_1_4_2_3 +8 8 4 all_1_4_2_3 +9 9 4 all_1_4_2_3 diff --git a/tests/queries/0_stateless/02668_column_block_number.sql b/tests/queries/0_stateless/02668_column_block_number.sql new file mode 100644 index 00000000000..4d4d54bba5f --- /dev/null +++ b/tests/queries/0_stateless/02668_column_block_number.sql @@ -0,0 +1,32 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test (id UInt32, a UInt32) ENGINE = MergeTree ORDER BY id SETTINGS allow_experimental_block_number_column = true; + +INSERT INTO test(id,a) VALUES (1,1),(2,2),(3,3); +INSERT INTO test(id,a) VALUES (4,4),(5,5),(6,6); + +SELECT '*** BEFORE MUTATION BEFORE MERGE ***'; +SELECT id,a,_block_number,_part from test ORDER BY id; + +set mutations_sync=1; +ALTER TABLE test UPDATE a=0 WHERE id<4; + +SELECT '*** AFTER MUTATION BEFORE MERGE ***'; +SELECT id,a,_block_number,_part from test ORDER BY id; + +OPTIMIZE TABLE test FINAL; + +SELECT '*** AFTER MUTATION AFTER MERGE ***'; +SELECT *,_block_number,_part from test ORDER BY id; + +INSERT INTO test(id,a) VALUES (7,7),(8,8),(9,9); + +SELECT '*** AFTER MUTATION AFTER MERGE , NEW BLOCK ***'; +SELECT *,_block_number,_part from test ORDER BY id; + +OPTIMIZE TABLE test FINAL; + +SELECT '*** AFTER MUTATION AFTER MERGE , NEW BLOCK MERGED ***'; +SELECT *,_block_number,_part from test ORDER BY id; + +DROP TABLE test; \ No newline at end of file diff --git a/tests/queries/0_stateless/02668_column_block_number_vertical_merge.reference b/tests/queries/0_stateless/02668_column_block_number_vertical_merge.reference new file mode 100644 index 00000000000..f08220243ff --- /dev/null +++ b/tests/queries/0_stateless/02668_column_block_number_vertical_merge.reference @@ -0,0 +1,41 @@ +*** BEFORE MUTATION BEFORE MERGE *** +1 1 1 all_1_1_0 +2 2 1 all_1_1_0 +3 3 1 all_1_1_0 +4 4 2 all_2_2_0 +5 5 2 all_2_2_0 +6 6 2 all_2_2_0 +*** AFTER MUTATION BEFORE MERGE *** +1 0 1 all_1_1_0_3 +2 0 1 all_1_1_0_3 +3 0 1 all_1_1_0_3 +4 4 2 all_2_2_0_3 +5 5 2 all_2_2_0_3 +6 6 2 all_2_2_0_3 +*** AFTER MUTATION AFTER MERGE *** +1 0 1 all_1_2_1_3 +2 0 1 all_1_2_1_3 +3 0 1 all_1_2_1_3 +4 4 2 all_1_2_1_3 +5 5 2 all_1_2_1_3 +6 6 2 all_1_2_1_3 +*** AFTER MUTATION AFTER MERGE , NEW BLOCK *** +1 0 1 all_1_2_1_3 +2 0 1 all_1_2_1_3 +3 0 1 all_1_2_1_3 +4 4 2 all_1_2_1_3 +5 5 2 all_1_2_1_3 +6 6 2 all_1_2_1_3 +7 7 4 all_4_4_0 +8 8 4 all_4_4_0 +9 9 4 all_4_4_0 +*** AFTER MUTATION AFTER MERGE , NEW BLOCK MERGED *** +1 0 1 all_1_4_2_3 +2 0 1 all_1_4_2_3 +3 0 1 all_1_4_2_3 +4 4 2 all_1_4_2_3 +5 5 2 all_1_4_2_3 +6 6 2 all_1_4_2_3 +7 7 4 all_1_4_2_3 +8 8 4 all_1_4_2_3 +9 9 4 all_1_4_2_3 diff --git a/tests/queries/0_stateless/02668_column_block_number_vertical_merge.sql b/tests/queries/0_stateless/02668_column_block_number_vertical_merge.sql new file mode 100644 index 00000000000..a32060efae1 --- /dev/null +++ b/tests/queries/0_stateless/02668_column_block_number_vertical_merge.sql @@ -0,0 +1,36 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test (id UInt32, a UInt32) ENGINE = MergeTree ORDER BY id SETTINGS allow_experimental_block_number_column = true, + vertical_merge_algorithm_min_rows_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 0, + min_rows_for_wide_part = 1, + min_bytes_for_wide_part = 1; + +INSERT INTO test(id,a) VALUES (1,1),(2,2),(3,3); +INSERT INTO test(id,a) VALUES (4,4),(5,5),(6,6); + +SELECT '*** BEFORE MUTATION BEFORE MERGE ***'; +SELECT id,a,_block_number,_part from test ORDER BY id; + +set mutations_sync=1; +ALTER TABLE test UPDATE a=0 WHERE id<4; + +SELECT '*** AFTER MUTATION BEFORE MERGE ***'; +SELECT id,a,_block_number,_part from test ORDER BY id; + +OPTIMIZE TABLE test FINAL; + +SELECT '*** AFTER MUTATION AFTER MERGE ***'; +SELECT *,_block_number,_part from test ORDER BY id; + +INSERT INTO test(id,a) VALUES (7,7),(8,8),(9,9); + +SELECT '*** AFTER MUTATION AFTER MERGE , NEW BLOCK ***'; +SELECT *,_block_number,_part from test ORDER BY id; + +OPTIMIZE TABLE test FINAL; + +SELECT '*** AFTER MUTATION AFTER MERGE , NEW BLOCK MERGED ***'; +SELECT *,_block_number,_part from test ORDER BY id; + +DROP TABLE test; \ No newline at end of file diff --git a/tests/queries/0_stateless/02668_column_block_number_with_projections.reference b/tests/queries/0_stateless/02668_column_block_number_with_projections.reference new file mode 100644 index 00000000000..289a21035b5 --- /dev/null +++ b/tests/queries/0_stateless/02668_column_block_number_with_projections.reference @@ -0,0 +1,19 @@ +0 +1 +2 +3 +*** AFTER FIRST OPTIMIZE *** +0 1 +1 2 +1 2 +2 3 +3 3 +*** AFTER SECOND OPTIMIZE *** +0 1 +1 2 +1 2 +2 3 +3 3 +4 4 +5 4 +6 4 diff --git a/tests/queries/0_stateless/02668_column_block_number_with_projections.sql b/tests/queries/0_stateless/02668_column_block_number_with_projections.sql new file mode 100644 index 00000000000..25ec411967b --- /dev/null +++ b/tests/queries/0_stateless/02668_column_block_number_with_projections.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS t; +CREATE TABLE t (x UInt8, PROJECTION p (SELECT x GROUP BY x)) ENGINE = MergeTree ORDER BY () SETTINGS allow_experimental_block_number_column=true; +INSERT INTO t VALUES (0); +INSERT INTO t VALUES (1),(1); +INSERT INTO t VALUES (2),(3); + +SELECT x FROM t GROUP BY x; +OPTIMIZE TABLE t FINAL; + +SELECT '*** AFTER FIRST OPTIMIZE ***'; +SELECT x,_block_number FROM t; + +INSERT INTO t VALUES (4), (5), (6); +OPTIMIZE TABLE t FINAL; +SELECT '*** AFTER SECOND OPTIMIZE ***'; +SELECT x,_block_number FROM t; + +DROP TABLE t; \ No newline at end of file From 06513f60ebf548f69f47c4dde649ff30bea6b971 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 20 Sep 2023 15:16:39 +0200 Subject: [PATCH 92/93] Revert "refine error code of duplicated index in create query" --- src/Interpreters/InterpreterCreateQuery.cpp | 4 ---- .../0_stateless/02884_duplicate_index_name.reference | 0 .../queries/0_stateless/02884_duplicate_index_name.sql | 10 ---------- 3 files changed, 14 deletions(-) delete mode 100644 tests/queries/0_stateless/02884_duplicate_index_name.reference delete mode 100644 tests/queries/0_stateless/02884_duplicate_index_name.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 3654f307eb9..a0635f18214 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -96,7 +96,6 @@ namespace ErrorCodes extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY; extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE; extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_INDEX; extern const int LOGICAL_ERROR; extern const int UNKNOWN_DATABASE; extern const int PATH_ACCESS_DENIED; @@ -699,8 +698,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti for (const auto & index : create.columns_list->indices->children) { IndexDescription index_desc = IndexDescription::getIndexFromAST(index->clone(), properties.columns, getContext()); - if (properties.indices.has(index_desc.name)) - throw Exception(ErrorCodes::ILLEGAL_INDEX, "Duplicated index name {}", backQuoteIfNeed(index_desc.name)); const auto & settings = getContext()->getSettingsRef(); if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index) { @@ -715,7 +712,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti properties.indices.push_back(index_desc); } - if (create.columns_list->projections) for (const auto & projection_ast : create.columns_list->projections->children) { diff --git a/tests/queries/0_stateless/02884_duplicate_index_name.reference b/tests/queries/0_stateless/02884_duplicate_index_name.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02884_duplicate_index_name.sql b/tests/queries/0_stateless/02884_duplicate_index_name.sql deleted file mode 100644 index 4cd9ae6d2a2..00000000000 --- a/tests/queries/0_stateless/02884_duplicate_index_name.sql +++ /dev/null @@ -1,10 +0,0 @@ -DROP TABLE IF EXISTS test_dup_index; - -CREATE TABLE test_dup_index -( - a Int64, - b Int64, - INDEX idx_a a TYPE minmax, - INDEX idx_a b TYPE minmax -) Engine = MergeTree() -ORDER BY a; -- { serverError ILLEGAL_INDEX } From c75f7c843456fe184f1d7d5e40e77d27123a441b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 20 Sep 2023 13:27:11 +0000 Subject: [PATCH 93/93] Correct merge result --- src/Common/ErrorCodes.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 95614c7e9ce..be2b0a7bd5e 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -585,8 +585,9 @@ M(700, USER_SESSION_LIMIT_EXCEEDED) \ M(701, CLUSTER_DOESNT_EXIST) \ M(702, CLIENT_INFO_DOES_NOT_MATCH) \ - M(703, CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS) \ - M(704, INVALID_IDENTIFIER) \ + M(703, INVALID_IDENTIFIER) \ + M(704, CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS) \ + \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \