From 3fe2e1084bb3a7cc0def8b9b720b32455d2f10e7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 5 Sep 2023 22:25:34 +0000 Subject: [PATCH 01/56] 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/56] 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/56] 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/56] 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/56] 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/56] 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/56] 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 3a78ba6de4acd52ea22f6a9ea6757f7b740344a7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 9 Sep 2023 21:36:39 +0000 Subject: [PATCH 08/56] 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 09/56] 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 10/56] 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 11/56] 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 12/56] 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 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 13/56] 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 14/56] 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 15/56] 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 16/56] 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 17/56] 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 18/56] 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 8ef910e1fa0c4d8d6053b111efab5b4aecaaffb2 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 15 Sep 2023 22:34:09 +0200 Subject: [PATCH 19/56] 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 46fa7dbb8080aa9a2189a1fe0adbe3110f89c11f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 5 Sep 2023 12:59:32 +0000 Subject: [PATCH 20/56] 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 21/56] 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 22/56] 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 23/56] 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 24/56] 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 25/56] 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 26/56] 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 27/56] 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 28/56] 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 29/56] 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 30/56] 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 31/56] 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 32/56] 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 33/56] 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 326eaa9bc583bd0397c97760580679b5b50d46c6 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 18 Sep 2023 18:26:53 +0200 Subject: [PATCH 34/56] 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 142a4631f14f02ad198e9c67700f44ec1e001dae Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 19 Sep 2023 11:22:10 +0200 Subject: [PATCH 35/56] 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 36/56] 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 37/56] 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 363ee7cae58c4118452bd856406d55469dae9e94 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Sep 2023 13:41:55 +0200 Subject: [PATCH 38/56] 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 39/56] 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 40/56] 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 27d8eefe9376ba73ee8a898ec674d324b1a91d7c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 19 Sep 2023 17:14:32 +0300 Subject: [PATCH 41/56] 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 b1b49f430af91868d9da8cb0a34aa5fa58e0bc3b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 19 Sep 2023 15:32:58 +0000 Subject: [PATCH 42/56] 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 43/56] 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 44/56] 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 45/56] 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 fa51399ede28f551a62061d6961032d1c99e763a Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 19 Sep 2023 21:39:59 +0000 Subject: [PATCH 46/56] 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 47/56] 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 48/56] 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 49/56] 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 50/56] 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 51/56] 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 52/56] 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 53/56] 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 54/56] 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 55/56] 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 c75f7c843456fe184f1d7d5e40e77d27123a441b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 20 Sep 2023 13:27:11 +0000 Subject: [PATCH 56/56] 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) \