From 3fe2e1084bb3a7cc0def8b9b720b32455d2f10e7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 5 Sep 2023 22:25:34 +0000 Subject: [PATCH 01/45] 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/45] 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/45] 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/45] 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/45] 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/45] 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 ffa82e9297c1d01f77793cc7b43aa4fb7bbec9c4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Sep 2023 16:58:13 +0200 Subject: [PATCH 07/45] Fix filtering parts with indexHint for non analyzer Signed-off-by: Azat Khuzhin --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 5 ++++- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 3 +++ tests/analyzer_tech_debt.txt | 1 + .../0_stateless/02880_indexHint__partition_id.reference | 9 +++++++++ .../0_stateless/02880_indexHint__partition_id.sql | 9 +++++++++ 5 files changed, 26 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02880_indexHint__partition_id.reference create mode 100644 tests/queries/0_stateless/02880_indexHint__partition_id.sql diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 4b95b74c3af..32802d5fa02 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1271,7 +1271,10 @@ static void buildIndexes( } /// TODO Support row_policy_filter and additional_filters - indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context); + if (settings.allow_experimental_analyzer) + indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context); + else + indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, query_info.query, context); indexes->use_skip_indexes = settings.use_skip_indexes; bool final = query_info.isFinal(); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 31aa2dbb61f..9c93ecb14c9 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -814,6 +814,9 @@ std::optional> MergeTreeDataSelectExecutor::filterPar ASTPtr expression_ast; auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */); + if (virtual_columns_block.rows() == 0) + return {}; + // Generate valid expressions for filtering VirtualColumnUtils::prepareFilterBlockWithQuery(query, context, virtual_columns_block, expression_ast); diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 4419190e12c..652ab0b99de 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -54,6 +54,7 @@ 01710_projection_additional_filters 01721_join_implicit_cast_long 01739_index_hint +02880_indexHint__partition_id 01747_join_view_filter_dictionary 01748_partition_id_pruning 01756_optimize_skip_unused_shards_rewrite_in diff --git a/tests/queries/0_stateless/02880_indexHint__partition_id.reference b/tests/queries/0_stateless/02880_indexHint__partition_id.reference new file mode 100644 index 00000000000..365e7b676c7 --- /dev/null +++ b/tests/queries/0_stateless/02880_indexHint__partition_id.reference @@ -0,0 +1,9 @@ +-- { echoOn } +select * from data prewhere indexHint(_partition_id = '1'); +1 +select count() from data prewhere indexHint(_partition_id = '1'); +1 +select * from data where indexHint(_partition_id = '1'); +1 +select count() from data where indexHint(_partition_id = '1'); +1 diff --git a/tests/queries/0_stateless/02880_indexHint__partition_id.sql b/tests/queries/0_stateless/02880_indexHint__partition_id.sql new file mode 100644 index 00000000000..d15b3f4ccea --- /dev/null +++ b/tests/queries/0_stateless/02880_indexHint__partition_id.sql @@ -0,0 +1,9 @@ +drop table if exists data; +create table data (part Int) engine=MergeTree() order by tuple() partition by part; +insert into data values (1)(2); + +-- { echoOn } +select * from data prewhere indexHint(_partition_id = '1'); +select count() from data prewhere indexHint(_partition_id = '1'); +select * from data where indexHint(_partition_id = '1'); +select count() from data where indexHint(_partition_id = '1'); From 3a78ba6de4acd52ea22f6a9ea6757f7b740344a7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 9 Sep 2023 21:36:39 +0000 Subject: [PATCH 08/45] 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/45] 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/45] 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/45] 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/45] Fix build: remove unused variables --- src/Interpreters/Context.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7834f5893d3..526a644ce6f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4634,13 +4634,11 @@ bool Context::canUseParallelReplicas() const bool Context::canUseParallelReplicasOnInitiator() const { - const auto & settings_ref = getSettingsRef(); return canUseParallelReplicas() && !getClientInfo().collaborate_with_initiator; } bool Context::canUseParallelReplicasOnFollower() const { - const auto & settings_ref = getSettingsRef(); return canUseParallelReplicas() && getClientInfo().collaborate_with_initiator; } From ca57eb95466b3532bae295fda9e42004d9e88301 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Thu, 14 Sep 2023 15:55:37 +0200 Subject: [PATCH 13/45] Fix addData function --- src/Functions/FunctionsOpDate.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/FunctionsOpDate.cpp b/src/Functions/FunctionsOpDate.cpp index 0d8ca2b58cc..dcbc1fc1256 100644 --- a/src/Functions/FunctionsOpDate.cpp +++ b/src/Functions/FunctionsOpDate.cpp @@ -53,7 +53,6 @@ public: } bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 2}; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { From 7643e216bead8c98d742427869cd121655cbdd28 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Thu, 14 Sep 2023 15:58:00 +0200 Subject: [PATCH 14/45] Better test --- tests/queries/0_stateless/02834_add_sub_date_functions.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02834_add_sub_date_functions.sql b/tests/queries/0_stateless/02834_add_sub_date_functions.sql index 44d9bb8a2aa..9c77ba7e378 100644 --- a/tests/queries/0_stateless/02834_add_sub_date_functions.sql +++ b/tests/queries/0_stateless/02834_add_sub_date_functions.sql @@ -1,6 +1,6 @@ SET session_timezone = 'UTC'; -SELECT ADDDATE('2022-05-07'::Date, INTERVAL 5 MINUTE); +SELECT ADDDATE(materialize('2022-05-07'::Date), INTERVAL 5 MINUTE); SELECT addDate('2022-05-07'::Date, INTERVAL 5 MINUTE); SELECT addDate('2022-05-07'::Date32, INTERVAL 5 MINUTE); @@ -14,7 +14,7 @@ SELECT addDate('1234', INTERVAL 5 MINUTE); -- { serverError ILLEGAL_TYPE_OF_ARG SELECT '---'; -SELECT SUBDATE('2022-05-07'::Date, INTERVAL 5 MINUTE); +SELECT SUBDATE(materialize('2022-05-07'::Date), INTERVAL 5 MINUTE); SELECT subDate('2022-05-07'::Date, INTERVAL 5 MINUTE); SELECT subDate('2022-05-07'::Date32, INTERVAL 5 MINUTE); From 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 15/45] 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 16/45] 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 46fa7dbb8080aa9a2189a1fe0adbe3110f89c11f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 5 Sep 2023 12:59:32 +0000 Subject: [PATCH 17/45] 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 18/45] 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 19/45] 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 20/45] 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 21/45] 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 22/45] 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 23/45] 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 24/45] 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 25/45] 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 26/45] 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 27/45] 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 28/45] 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 29/45] 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 30/45] 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 94d737264cb62016cef3d89dc3c73fce6a5c9fd8 Mon Sep 17 00:00:00 2001 From: priera Date: Thu, 31 Aug 2023 15:37:48 +0200 Subject: [PATCH 31/45] inserting non-duplicate chunks --- src/Processors/Sinks/IOutputChunkGenerator.h | 22 +++++ src/Processors/Sinks/OutputChunkGenerator.cpp | 80 +++++++++++++++++++ src/Processors/Sinks/SinkToStorage.cpp | 13 ++- src/Processors/Sinks/SinkToStorage.h | 7 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 24 ++---- .../MergeTree/ReplicatedMergeTreeSink.h | 12 --- ...erialized_views_duplicated_parts.reference | 14 ++++ ...rt_materialized_views_duplicated_parts.sql | 44 ++++++++++ 8 files changed, 179 insertions(+), 37 deletions(-) create mode 100644 src/Processors/Sinks/IOutputChunkGenerator.h create mode 100644 src/Processors/Sinks/OutputChunkGenerator.cpp create mode 100644 tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.reference create mode 100644 tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.sql diff --git a/src/Processors/Sinks/IOutputChunkGenerator.h b/src/Processors/Sinks/IOutputChunkGenerator.h new file mode 100644 index 00000000000..3a92368df14 --- /dev/null +++ b/src/Processors/Sinks/IOutputChunkGenerator.h @@ -0,0 +1,22 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class IOutputChunkGenerator { +public: + static std::unique_ptr createCopyRanges(ContextPtr context); + static std::unique_ptr createDefault(); + + virtual ~IOutputChunkGenerator() = default; + + virtual void onNewChunkArrived(Chunk chunk) = 0; + virtual void onRowsProcessed(size_t row_count, bool append) = 0; + + virtual Chunk generateChunk() = 0; +}; + +} diff --git a/src/Processors/Sinks/OutputChunkGenerator.cpp b/src/Processors/Sinks/OutputChunkGenerator.cpp new file mode 100644 index 00000000000..dab5adb1792 --- /dev/null +++ b/src/Processors/Sinks/OutputChunkGenerator.cpp @@ -0,0 +1,80 @@ +#include + +namespace DB { + +class ForwardEverythingGenerator : public IOutputChunkGenerator { +public: + + explicit ForwardEverythingGenerator() = default; + + void onNewChunkArrived(Chunk chunk) override + { + in_chunk = chunk.clone(); + } + + void onRowsProcessed(size_t /*row_count*/, bool /*append*/) override + {} + + Chunk generateChunk() override + { + return std::move(in_chunk); + } + +private: + Chunk in_chunk; +}; + +class CopyRangesGenerator : public IOutputChunkGenerator { +public: + explicit CopyRangesGenerator() = default; + + void onNewChunkArrived(Chunk chunk) override + { + out_cols = chunk.cloneEmptyColumns(); + in_chunk = std::move(chunk); + } + + void onRowsProcessed(size_t row_count, bool append) override + { + if (append) + { + const Columns& in_cols = in_chunk.getColumns(); + for (size_t i = 0; i < out_cols.size(); i++) + { + out_cols[i]->insertRangeFrom(*(in_cols[i]), row_offset, row_count); + } + final_chunk_rows += row_count; + } + + row_offset += row_count; + } + + Chunk generateChunk() override + { + return Chunk(std::move(out_cols), final_chunk_rows); + } + +private: + Chunk in_chunk; + MutableColumns out_cols; + size_t row_offset = 0; + size_t final_chunk_rows = 0; +}; + +std::unique_ptr IOutputChunkGenerator::createCopyRanges(ContextPtr context) +{ + // If MV is responsible for deduplication, block is not considered duplicated. + if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) + { + return createDefault(); + } + + return std::make_unique(); +} + +std::unique_ptr IOutputChunkGenerator::createDefault() +{ + return std::make_unique(); +} + +} diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index 5f9f9f9b1a1..84743306446 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -4,7 +4,12 @@ namespace DB { -SinkToStorage::SinkToStorage(const Block & header) : ExceptionKeepingTransform(header, header, false) {} +SinkToStorage::SinkToStorage(const Block & header) : SinkToStorage(header, IOutputChunkGenerator::createDefault()) {} + +SinkToStorage::SinkToStorage(const Block & header, std::unique_ptr output_generator_) + : ExceptionKeepingTransform(header, header, false), + output_generator(std::move(output_generator_)) +{ } void SinkToStorage::onConsume(Chunk chunk) { @@ -15,15 +20,15 @@ void SinkToStorage::onConsume(Chunk chunk) */ Nested::validateArraySizes(getHeader().cloneWithColumns(chunk.getColumns())); + output_generator->onNewChunkArrived(chunk.clone()); consume(chunk.clone()); - if (!lastBlockIsDuplicate()) - cur_chunk = std::move(chunk); } SinkToStorage::GenerateResult SinkToStorage::onGenerate() { GenerateResult res; - res.chunk = std::move(cur_chunk); + + res.chunk = output_generator->generateChunk(); res.is_done = true; return res; } diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index 023bbd8b094..e810578f651 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { @@ -13,13 +14,15 @@ friend class PartitionedSink; public: explicit SinkToStorage(const Block & header); + explicit SinkToStorage(const Block & header, std::unique_ptr output_chunk_generator); const Block & getHeader() const { return inputs.front().getHeader(); } void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } protected: virtual void consume(Chunk chunk) = 0; - virtual bool lastBlockIsDuplicate() const { return false; } + + IOutputChunkGenerator& getOutputGenerator() { return *output_generator; } private: std::vector table_locks; @@ -27,7 +30,7 @@ private: void onConsume(Chunk chunk) override; GenerateResult onGenerate() override; - Chunk cur_chunk; + std::unique_ptr output_generator; }; using SinkToStoragePtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 75679a5750a..9fc7492a7fe 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -130,7 +130,7 @@ ReplicatedMergeTreeSinkImpl::ReplicatedMergeTreeSinkImpl( bool majority_quorum, ContextPtr context_, bool is_attach_) - : SinkToStorage(metadata_snapshot_->getSampleBlock()) + : SinkToStorage(metadata_snapshot_->getSampleBlock(), IOutputChunkGenerator::createCopyRanges(context_)) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , required_quorum_size(majority_quorum ? std::nullopt : std::make_optional(quorum_size)) @@ -386,13 +386,7 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) finishDelayedChunk(zookeeper); delayed_chunk = std::make_unique(); delayed_chunk->partitions = std::move(partitions); - - /// If deduplicated data should not be inserted into MV, we need to set proper - /// value for `last_block_is_duplicate`, which is possible only after the part is committed. - /// Othervide we can delay commit. - /// TODO: we can also delay commit if there is no MVs. - if (!settings.deduplicate_blocks_in_dependent_materialized_views) - finishDelayedChunk(zookeeper); + finishDelayedChunk(zookeeper); ++num_blocks_processed; } @@ -403,8 +397,6 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF if (!delayed_chunk) return; - last_block_is_duplicate = false; - for (auto & partition : delayed_chunk->partitions) { ProfileEventsScope scoped_attach(&partition.part_counters); @@ -415,9 +407,10 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF try { - bool deduplicated = commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num, false).second; + const size_t rowsCount = partition.temp_part.part->rows_count; + const bool deduplicated = commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num, false).second; - last_block_is_duplicate = last_block_is_duplicate || deduplicated; + getOutputGenerator().onRowsProcessed(rowsCount, !deduplicated); /// Set a special error code if the block is duplicate int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; @@ -1092,13 +1085,6 @@ void ReplicatedMergeTreeSinkImpl::onStart() storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event, context, true); } -template -void ReplicatedMergeTreeSinkImpl::onFinish() -{ - auto zookeeper = storage.getZooKeeper(); - finishDelayedChunk(std::make_shared(zookeeper)); -} - template void ReplicatedMergeTreeSinkImpl::waitForQuorum( const ZooKeeperWithFaultInjectionPtr & zookeeper, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 4a192a822f5..b208154631c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -51,23 +51,12 @@ public: void onStart() override; void consume(Chunk chunk) override; - void onFinish() override; String getName() const override { return "ReplicatedMergeTreeSink"; } /// For ATTACHing existing data on filesystem. bool writeExistingPart(MergeTreeData::MutableDataPartPtr & part); - /// For proper deduplication in MaterializedViews - bool lastBlockIsDuplicate() const override - { - /// If MV is responsible for deduplication, block is not considered duplicating. - if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) - return false; - - return last_block_is_duplicate; - } - struct DelayedChunk; private: using BlockIDsType = std::conditional_t, String>; @@ -122,7 +111,6 @@ private: bool is_attach = false; bool quorum_parallel = false; const bool deduplicate = true; - bool last_block_is_duplicate = false; UInt64 num_blocks_processed = 0; using Logger = Poco::Logger; diff --git a/tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.reference b/tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.reference new file mode 100644 index 00000000000..325f639813a --- /dev/null +++ b/tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.reference @@ -0,0 +1,14 @@ +Initial +2020-01-01 13:00:00 24 +Last block is duplicate +2020-01-01 13:00:00 24 +2021-09-01 11:00:00 24 +One block is duplicate (default setting) +2020-01-01 13:00:00 24 +2021-09-01 11:00:00 24 +2022-01-01 12:00:00 24 +One block is duplicate (changed setting) +2020-01-01 13:00:00 24 +2021-09-01 11:00:00 24 +2022-01-01 12:00:00 24 +2023-01-01 12:00:00 24 diff --git a/tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.sql b/tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.sql new file mode 100644 index 00000000000..c087e826a13 --- /dev/null +++ b/tests/queries/0_stateless/02869_insert_materialized_views_duplicated_parts.sql @@ -0,0 +1,44 @@ +-- Tags: zookeeper + +DROP TABLE IF EXISTS landing SYNC; +DROP TABLE IF EXISTS mv SYNC; + +CREATE TABLE landing +( + `time` DateTime, + `number` Int64 +) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{database}/tables/landing/', 'r1') +PARTITION BY toYYYYMMDD(time) +ORDER BY time; + +CREATE MATERIALIZED VIEW mv +ENGINE = ReplicatedSummingMergeTree('/clickhouse/{database}/tables/mv', 'r1') +PARTITION BY toYYYYMMDD(hour) ORDER BY hour +AS SELECT + toStartOfHour(time) AS hour, + sum(number) AS sum_amount + FROM landing GROUP BY hour; + +SELECT 'Initial'; +INSERT INTO landing VALUES ('2020-01-01 13:23:34', 24); +SELECT * FROM mv ORDER BY hour; + +SELECT 'Last block is duplicate'; +INSERT INTO landing VALUES ('2021-09-01 11:00:00', 24), ('2020-01-01 13:23:34', 24); +SELECT * FROM mv ORDER BY hour; + +SELECT 'One block is duplicate (default setting)'; +SET max_insert_delayed_streams_for_parallel_write = 0; +INSERT INTO landing VALUES ('2021-09-01 11:00:00', 24), ('2022-01-01 12:03:00', 24); +SELECT * FROM mv ORDER BY hour; + +SELECT 'One block is duplicate (changed setting)'; +SET max_insert_delayed_streams_for_parallel_write = 5; +INSERT INTO landing VALUES ('2021-09-01 11:00:00', 24), ('2023-01-01 12:03:00', 24); + +SELECT * FROM mv ORDER BY hour; + +DROP TABLE mv; +DROP TABLE landing; + From 6d27ea2742a162351cf1f1738523949a6e7d4ce1 Mon Sep 17 00:00:00 2001 From: priera Date: Tue, 12 Sep 2023 12:49:43 +0200 Subject: [PATCH 32/45] fixed style --- src/Processors/Sinks/IOutputChunkGenerator.h | 3 ++- src/Processors/Sinks/OutputChunkGenerator.cpp | 9 ++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Processors/Sinks/IOutputChunkGenerator.h b/src/Processors/Sinks/IOutputChunkGenerator.h index 3a92368df14..b5e1d76af26 100644 --- a/src/Processors/Sinks/IOutputChunkGenerator.h +++ b/src/Processors/Sinks/IOutputChunkGenerator.h @@ -6,7 +6,8 @@ namespace DB { -class IOutputChunkGenerator { +class IOutputChunkGenerator +{ public: static std::unique_ptr createCopyRanges(ContextPtr context); static std::unique_ptr createDefault(); diff --git a/src/Processors/Sinks/OutputChunkGenerator.cpp b/src/Processors/Sinks/OutputChunkGenerator.cpp index dab5adb1792..eb161c0be1e 100644 --- a/src/Processors/Sinks/OutputChunkGenerator.cpp +++ b/src/Processors/Sinks/OutputChunkGenerator.cpp @@ -1,8 +1,10 @@ #include -namespace DB { +namespace DB +{ -class ForwardEverythingGenerator : public IOutputChunkGenerator { +class ForwardEverythingGenerator : public IOutputChunkGenerator +{ public: explicit ForwardEverythingGenerator() = default; @@ -24,7 +26,8 @@ private: Chunk in_chunk; }; -class CopyRangesGenerator : public IOutputChunkGenerator { +class CopyRangesGenerator : public IOutputChunkGenerator +{ public: explicit CopyRangesGenerator() = default; From b30e33580b54349952b7f52813a3011efbe509a9 Mon Sep 17 00:00:00 2001 From: priera Date: Tue, 12 Sep 2023 17:12:43 +0200 Subject: [PATCH 33/45] fix test --- src/Processors/Sinks/OutputChunkGenerator.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Sinks/OutputChunkGenerator.cpp b/src/Processors/Sinks/OutputChunkGenerator.cpp index eb161c0be1e..107c21b2319 100644 --- a/src/Processors/Sinks/OutputChunkGenerator.cpp +++ b/src/Processors/Sinks/OutputChunkGenerator.cpp @@ -35,6 +35,8 @@ public: { out_cols = chunk.cloneEmptyColumns(); in_chunk = std::move(chunk); + row_offset = 0; + final_chunk_rows = 0; } void onRowsProcessed(size_t row_count, bool append) override From 663ce74609170031d22a330c3478569f7723fea0 Mon Sep 17 00:00:00 2001 From: priera Date: Wed, 13 Sep 2023 14:39:29 +0200 Subject: [PATCH 34/45] fixed clang-tidy error --- src/Processors/Sinks/SinkToStorage.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index e810578f651..dc7ba23b52a 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -14,7 +14,7 @@ friend class PartitionedSink; public: explicit SinkToStorage(const Block & header); - explicit SinkToStorage(const Block & header, std::unique_ptr output_chunk_generator); + explicit SinkToStorage(const Block & header, std::unique_ptr output_generator_); const Block & getHeader() const { return inputs.front().getHeader(); } void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } From b92e37d2dfe0c2088429e1c90fbe73e9f384aa3b Mon Sep 17 00:00:00 2001 From: priera Date: Tue, 19 Sep 2023 12:13:39 +0200 Subject: [PATCH 35/45] Fixing PR comments --- src/Processors/Sinks/IOutputChunkGenerator.h | 5 ++++- src/Processors/Sinks/OutputChunkGenerator.cpp | 20 ++++++++++++------- .../MergeTree/ReplicatedMergeTreeSink.cpp | 3 ++- 3 files changed, 19 insertions(+), 9 deletions(-) diff --git a/src/Processors/Sinks/IOutputChunkGenerator.h b/src/Processors/Sinks/IOutputChunkGenerator.h index b5e1d76af26..824313a2394 100644 --- a/src/Processors/Sinks/IOutputChunkGenerator.h +++ b/src/Processors/Sinks/IOutputChunkGenerator.h @@ -6,10 +6,13 @@ namespace DB { +/// This interface is meant to be used by the SinkToStorage processor +/// SinkToStorage delegates on it the creation of the data chunk that will deliver to the next stages of the query pipeline +/// Default implementation (createDefault() factory method) just forwards everything that it receives class IOutputChunkGenerator { public: - static std::unique_ptr createCopyRanges(ContextPtr context); + static std::unique_ptr createCopyRanges(bool deduplicate_later); static std::unique_ptr createDefault(); virtual ~IOutputChunkGenerator() = default; diff --git a/src/Processors/Sinks/OutputChunkGenerator.cpp b/src/Processors/Sinks/OutputChunkGenerator.cpp index 107c21b2319..942bf49a2d4 100644 --- a/src/Processors/Sinks/OutputChunkGenerator.cpp +++ b/src/Processors/Sinks/OutputChunkGenerator.cpp @@ -3,6 +3,7 @@ namespace DB { +/// Default implementation. The new chunk received is forwarded as-is to the next stages of the query class ForwardEverythingGenerator : public IOutputChunkGenerator { public: @@ -14,10 +15,10 @@ public: in_chunk = chunk.clone(); } - void onRowsProcessed(size_t /*row_count*/, bool /*append*/) override + void onRowsProcessed(size_t /*row_count*/, bool /*append*/) override {} - Chunk generateChunk() override + Chunk generateChunk() override { return std::move(in_chunk); } @@ -26,6 +27,10 @@ private: Chunk in_chunk; }; +/// Specific implementation which generates a chunk with just a subset of the rows received originally +/// Rows are assumed to be processed in the same order than they appear in the original chunk +/// Is up to the client to decide how many rows process at once, but after each range processed, +/// onRowsProcessed() has to be called, indicating whether append that range to the output chunk or not class CopyRangesGenerator : public IOutputChunkGenerator { public: @@ -39,7 +44,7 @@ public: final_chunk_rows = 0; } - void onRowsProcessed(size_t row_count, bool append) override + void onRowsProcessed(size_t row_count, bool append) override { if (append) { @@ -54,7 +59,7 @@ public: row_offset += row_count; } - Chunk generateChunk() override + Chunk generateChunk() override { return Chunk(std::move(out_cols), final_chunk_rows); } @@ -66,10 +71,11 @@ private: size_t final_chunk_rows = 0; }; -std::unique_ptr IOutputChunkGenerator::createCopyRanges(ContextPtr context) +std::unique_ptr IOutputChunkGenerator::createCopyRanges(bool deduplicate_later) { - // If MV is responsible for deduplication, block is not considered duplicated. - if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) + // If MV is responsible for deduplication, block won't be considered duplicated. + // So default implementation, forwarding all the data, is used + if (deduplicate_later) { return createDefault(); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 9fc7492a7fe..9cc1c6932c7 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -130,7 +130,8 @@ ReplicatedMergeTreeSinkImpl::ReplicatedMergeTreeSinkImpl( bool majority_quorum, ContextPtr context_, bool is_attach_) - : SinkToStorage(metadata_snapshot_->getSampleBlock(), IOutputChunkGenerator::createCopyRanges(context_)) + : SinkToStorage(metadata_snapshot_->getSampleBlock(), + IOutputChunkGenerator::createCopyRanges(context_->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views)) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , required_quorum_size(majority_quorum ? std::nullopt : std::make_optional(quorum_size)) From 8c29408f5eba3388a094fa937d19c251a5770791 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 7 Sep 2023 14:06:19 +0000 Subject: [PATCH 36/45] Parse data in JSON format as JSONEachRow if failed to parse metadata --- src/Formats/JSONUtils.cpp | 17 ++-- src/Formats/JSONUtils.h | 1 + .../Formats/Impl/JSONEachRowRowInputFormat.h | 3 +- .../Formats/Impl/JSONRowInputFormat.cpp | 82 ++++++++++++++----- .../Formats/Impl/JSONRowInputFormat.h | 18 +++- ..._as_json_each_row_on_no_metadata.reference | 3 + ...e_json_as_json_each_row_on_no_metadata.sql | 3 + 7 files changed, 98 insertions(+), 29 deletions(-) create mode 100644 tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.reference create mode 100644 tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.sql diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 6fbda869154..1a2849493c3 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -687,10 +687,9 @@ namespace JSONUtils return names_and_types; } - NamesAndTypesList readMetadataAndValidateHeader(ReadBuffer & in, const Block & header) + void validateMetadataByHeader(const NamesAndTypesList & names_and_types_from_metadata, const Block & header) { - auto names_and_types = JSONUtils::readMetadata(in); - for (const auto & [name, type] : names_and_types) + for (const auto & [name, type] : names_and_types_from_metadata) { if (!header.has(name)) continue; @@ -698,10 +697,16 @@ namespace JSONUtils auto header_type = header.getByName(name).type; if (!type->equals(*header_type)) throw Exception( - ErrorCodes::INCORRECT_DATA, - "Type {} of column '{}' from metadata is not the same as type in header {}", - type->getName(), name, header_type->getName()); + ErrorCodes::INCORRECT_DATA, + "Type {} of column '{}' from metadata is not the same as type in header {}", + type->getName(), name, header_type->getName()); } + } + + NamesAndTypesList readMetadataAndValidateHeader(ReadBuffer & in, const Block & header) + { + auto names_and_types = JSONUtils::readMetadata(in); + validateMetadataByHeader(names_and_types, header); return names_and_types; } diff --git a/src/Formats/JSONUtils.h b/src/Formats/JSONUtils.h index bd56eb646cb..27a357447f5 100644 --- a/src/Formats/JSONUtils.h +++ b/src/Formats/JSONUtils.h @@ -124,6 +124,7 @@ namespace JSONUtils NamesAndTypesList readMetadata(ReadBuffer & in); NamesAndTypesList readMetadataAndValidateHeader(ReadBuffer & in, const Block & header); + void validateMetadataByHeader(const NamesAndTypesList & names_and_types_from_metadata, const Block & header); bool skipUntilFieldInObject(ReadBuffer & in, const String & desired_field_name); void skipTheRestOfObject(ReadBuffer & in); diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h index d4246c37ea0..ad494d07fbc 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h @@ -32,10 +32,11 @@ public: String getName() const override { return "JSONEachRowRowInputFormat"; } void resetParser() override; -private: +protected: void readPrefix() override; void readSuffix() override; +private: bool readRow(MutableColumns & columns, RowReadExtension & ext) override; bool allowSyncAfterError() const override { return true; } void syncAfterError() override; diff --git a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp index ac7ba6048a5..a1507050fbc 100644 --- a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp @@ -12,42 +12,84 @@ namespace ErrorCodes } JSONRowInputFormat::JSONRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) - : JSONEachRowRowInputFormat(in_, header_, params_, format_settings_, false), validate_types_from_metadata(format_settings_.json.validate_types_from_metadata) + : JSONRowInputFormat(std::make_unique(in_), header_, params_, format_settings_) +{ +} + +JSONRowInputFormat::JSONRowInputFormat(std::unique_ptr buf, const DB::Block & header_, DB::IRowInputFormat::Params params_, const DB::FormatSettings & format_settings_) + : JSONEachRowRowInputFormat(*buf, header_, params_, format_settings_, false), validate_types_from_metadata(format_settings_.json.validate_types_from_metadata), peekable_buf(std::move(buf)) { } void JSONRowInputFormat::readPrefix() { - skipBOMIfExists(*in); - JSONUtils::skipObjectStart(*in); + skipBOMIfExists(*peekable_buf); + + PeekableReadBufferCheckpoint checkpoint(*peekable_buf); + NamesAndTypesList names_and_types_from_metadata; + + /// Try to parse metadata, if failed, try to parse data as JSONEachRow format. + try + { + JSONUtils::skipObjectStart(*peekable_buf); + names_and_types_from_metadata = JSONUtils::readMetadata(*peekable_buf); + JSONUtils::skipComma(*peekable_buf); + if (!JSONUtils::skipUntilFieldInObject(*peekable_buf, "data")) + throw Exception(ErrorCodes::INCORRECT_DATA, "Expected field \"data\" with table content"); + + JSONUtils::skipArrayStart(*peekable_buf); + data_in_square_brackets = true; + } + catch (...) + { + peekable_buf->rollbackToCheckpoint(); + JSONEachRowRowInputFormat::readPrefix(); + parse_as_json_each_row = true; + return; + } + if (validate_types_from_metadata) - JSONUtils::readMetadataAndValidateHeader(*in, getPort().getHeader()); - else - JSONUtils::readMetadata(*in); - - JSONUtils::skipComma(*in); - if (!JSONUtils::skipUntilFieldInObject(*in, "data")) - throw Exception(ErrorCodes::INCORRECT_DATA, "Expected field \"data\" with table content"); - - JSONUtils::skipArrayStart(*in); - data_in_square_brackets = true; + JSONUtils::validateMetadataByHeader(names_and_types_from_metadata, getPort().getHeader()); } void JSONRowInputFormat::readSuffix() { - JSONUtils::skipArrayEnd(*in); - JSONUtils::skipTheRestOfObject(*in); + if (parse_as_json_each_row) + { + JSONEachRowRowInputFormat::readSuffix(); + } + else + { + JSONUtils::skipArrayEnd(*peekable_buf); + JSONUtils::skipTheRestOfObject(*peekable_buf); + } } -JSONRowSchemaReader::JSONRowSchemaReader(ReadBuffer & in_) : ISchemaReader(in_) +JSONRowSchemaReader::JSONRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) + : JSONRowSchemaReader(std::make_unique(in_), format_settings_) +{ +} + +JSONRowSchemaReader::JSONRowSchemaReader(std::unique_ptr buf, const DB::FormatSettings & format_settings_) + : JSONEachRowSchemaReader(*buf, format_settings_), peekable_buf(std::move(buf)) { } NamesAndTypesList JSONRowSchemaReader::readSchema() { - skipBOMIfExists(in); - JSONUtils::skipObjectStart(in); - return JSONUtils::readMetadata(in); + skipBOMIfExists(*peekable_buf); + PeekableReadBufferCheckpoint checkpoint(*peekable_buf); + /// Try to parse metadata, if failed, try to parse data as JSONEachRow format + try + { + JSONUtils::skipObjectStart(*peekable_buf); + return JSONUtils::readMetadata(*peekable_buf); + } + catch (...) + { + peekable_buf->rollbackToCheckpoint(true); + return JSONEachRowSchemaReader::readSchema(); + } } void registerInputFormatJSON(FormatFactory & factory) @@ -69,7 +111,7 @@ void registerJSONSchemaReader(FormatFactory & factory) auto register_schema_reader = [&](const String & format) { factory.registerSchemaReader( - format, [](ReadBuffer & buf, const FormatSettings &) { return std::make_unique(buf); }); + format, [](ReadBuffer & buf, const FormatSettings & format_settings) { return std::make_unique(buf, format_settings); }); }; register_schema_reader("JSON"); /// JSONCompact has the same suffix with metadata. diff --git a/src/Processors/Formats/Impl/JSONRowInputFormat.h b/src/Processors/Formats/Impl/JSONRowInputFormat.h index 40b7cc2a268..5bdffe3dec0 100644 --- a/src/Processors/Formats/Impl/JSONRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowInputFormat.h @@ -24,20 +24,34 @@ public: String getName() const override { return "JSONRowInputFormat"; } private: + JSONRowInputFormat( + std::unique_ptr buf, + const Block & header_, + Params params_, + const FormatSettings & format_settings_); + void readPrefix() override; void readSuffix() override; const bool validate_types_from_metadata; + bool parse_as_json_each_row = false; + std::unique_ptr peekable_buf; + std::exception_ptr reading_metadata_exception; }; -class JSONRowSchemaReader : public ISchemaReader +class JSONRowSchemaReader : public JSONEachRowSchemaReader { public: - JSONRowSchemaReader(ReadBuffer & in_); + JSONRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_); NamesAndTypesList readSchema() override; bool hasStrictOrderOfColumns() const override { return false; } + +private: + JSONRowSchemaReader(std::unique_ptr buf, const FormatSettings & format_settings_); + + std::unique_ptr peekable_buf; }; } diff --git a/tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.reference b/tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.reference new file mode 100644 index 00000000000..9c61868a7dd --- /dev/null +++ b/tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.reference @@ -0,0 +1,3 @@ +a Nullable(Int64) +b Nullable(String) +10 Hello diff --git a/tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.sql b/tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.sql new file mode 100644 index 00000000000..ba7b720e3b6 --- /dev/null +++ b/tests/queries/0_stateless/02874_parse_json_as_json_each_row_on_no_metadata.sql @@ -0,0 +1,3 @@ +desc format(JSON, '{"a" : 10, "b" : "Hello"}'); +select * from format(JSON, '{"a" : 10, "b" : "Hello"}'); + From 5bd2e9f61043a30cb24f1f4221c50546d3426192 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 7 Sep 2023 16:57:25 +0000 Subject: [PATCH 37/45] Fix tests --- src/Processors/Formats/Impl/JSONRowInputFormat.cpp | 11 +++++++++++ src/Processors/Formats/Impl/JSONRowInputFormat.h | 3 +++ 2 files changed, 14 insertions(+) diff --git a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp index a1507050fbc..745c510be1a 100644 --- a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp @@ -65,6 +65,17 @@ void JSONRowInputFormat::readSuffix() } } +void JSONRowInputFormat::setReadBuffer(DB::ReadBuffer & in_) +{ + peekable_buf->setSubBuffer(in_); +} + +void JSONRowInputFormat::resetParser() +{ + JSONEachRowRowInputFormat::resetParser(); + peekable_buf->reset(); +} + JSONRowSchemaReader::JSONRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) : JSONRowSchemaReader(std::make_unique(in_), format_settings_) { diff --git a/src/Processors/Formats/Impl/JSONRowInputFormat.h b/src/Processors/Formats/Impl/JSONRowInputFormat.h index 5bdffe3dec0..771c0b64aee 100644 --- a/src/Processors/Formats/Impl/JSONRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowInputFormat.h @@ -23,6 +23,9 @@ public: String getName() const override { return "JSONRowInputFormat"; } + void setReadBuffer(ReadBuffer & in_) override; + void resetParser() override; + private: JSONRowInputFormat( std::unique_ptr buf, From f974970c3cb88474f053ce59808926e8c2f28a76 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 18 Sep 2023 18:41:04 +0000 Subject: [PATCH 38/45] Apply suggestion --- .../Formats/Impl/JSONRowInputFormat.cpp | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp index 745c510be1a..90537061723 100644 --- a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp @@ -40,16 +40,27 @@ void JSONRowInputFormat::readPrefix() JSONUtils::skipArrayStart(*peekable_buf); data_in_square_brackets = true; } - catch (...) + catch (const ParsingException &) + { + parse_as_json_each_row = true; + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::INCORRECT_DATA) + throw; + + parse_as_json_each_row = true; + } + + if (parse_as_json_each_row) { peekable_buf->rollbackToCheckpoint(); JSONEachRowRowInputFormat::readPrefix(); - parse_as_json_each_row = true; - return; } - - if (validate_types_from_metadata) + else if (validate_types_from_metadata) + { JSONUtils::validateMetadataByHeader(names_and_types_from_metadata, getPort().getHeader()); + } } void JSONRowInputFormat::readSuffix() From 003f5f77eb22731b86a316a2ab133fbfb920a84a Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 19 Sep 2023 15:01:28 +0200 Subject: [PATCH 39/45] refine error code of duplicated index in create query --- src/Interpreters/InterpreterCreateQuery.cpp | 4 ++++ .../0_stateless/02884_duplicate_index_name.reference | 0 .../queries/0_stateless/02884_duplicate_index_name.sql | 10 ++++++++++ 3 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/02884_duplicate_index_name.reference create mode 100644 tests/queries/0_stateless/02884_duplicate_index_name.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 1cd34c2a0f6..b426025413c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -95,6 +95,7 @@ namespace ErrorCodes extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY; extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE; extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_INDEX; extern const int LOGICAL_ERROR; extern const int UNKNOWN_DATABASE; extern const int PATH_ACCESS_DENIED; @@ -697,6 +698,8 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti for (const auto & index : create.columns_list->indices->children) { IndexDescription index_desc = IndexDescription::getIndexFromAST(index->clone(), properties.columns, getContext()); + if (properties.indices.has(index_desc.name)) + throw Exception(ErrorCodes::ILLEGAL_INDEX, "Duplicated index name {}", backQuoteIfNeed(index_desc.name)); const auto & settings = getContext()->getSettingsRef(); if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index) { @@ -711,6 +714,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti properties.indices.push_back(index_desc); } + if (create.columns_list->projections) for (const auto & projection_ast : create.columns_list->projections->children) { diff --git a/tests/queries/0_stateless/02884_duplicate_index_name.reference b/tests/queries/0_stateless/02884_duplicate_index_name.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02884_duplicate_index_name.sql b/tests/queries/0_stateless/02884_duplicate_index_name.sql new file mode 100644 index 00000000000..4647ab3702b --- /dev/null +++ b/tests/queries/0_stateless/02884_duplicate_index_name.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS test_dup_index; + +CREATE TABLE test_dup_index +( + a Int64, + b Int64, + INDEX idx_a a TYPE minmax, + INDEX idx_a b TYPE minmax +) Engine = MergeTree() +ORDER BY a; -- { serverError 127 } From 3e4eefc700d48284cd3c0bf788914dbcf7fd559a Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 19 Sep 2023 15:56:46 +0200 Subject: [PATCH 40/45] Update tests/queries/0_stateless/02884_duplicate_index_name.sql Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- tests/queries/0_stateless/02884_duplicate_index_name.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02884_duplicate_index_name.sql b/tests/queries/0_stateless/02884_duplicate_index_name.sql index 4647ab3702b..4cd9ae6d2a2 100644 --- a/tests/queries/0_stateless/02884_duplicate_index_name.sql +++ b/tests/queries/0_stateless/02884_duplicate_index_name.sql @@ -7,4 +7,4 @@ CREATE TABLE test_dup_index INDEX idx_a a TYPE minmax, INDEX idx_a b TYPE minmax ) Engine = MergeTree() -ORDER BY a; -- { serverError 127 } +ORDER BY a; -- { serverError ILLEGAL_INDEX } From 51b95abc2b9f4e8577d4376129b5c0c7f6b3fbca Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 19 Sep 2023 17:19:11 +0200 Subject: [PATCH 41/45] Update Settings.h (#54754) --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7a20c685b09..186d90d9f5d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -796,7 +796,7 @@ class IColumn; M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \ M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \ M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ - M(Bool, allow_experimental_undrop_table_query, false, "Allow to use undrop query to restore dropped table in a limited time", 0) \ + M(Bool, allow_experimental_undrop_table_query, true, "Allow to use undrop query to restore dropped table in a limited time", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ From b1b49f430af91868d9da8cb0a34aa5fa58e0bc3b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 19 Sep 2023 15:32:58 +0000 Subject: [PATCH 42/45] 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 c439c4bca212a784c752f791b146f71de8fa883a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 19 Sep 2023 21:39:21 +0200 Subject: [PATCH 43/45] Revert "Fix filtering parts with indexHint for non analyzer" --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 5 +---- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 3 --- tests/analyzer_tech_debt.txt | 1 - .../0_stateless/02880_indexHint__partition_id.reference | 9 --------- .../0_stateless/02880_indexHint__partition_id.sql | 9 --------- 5 files changed, 1 insertion(+), 26 deletions(-) delete mode 100644 tests/queries/0_stateless/02880_indexHint__partition_id.reference delete mode 100644 tests/queries/0_stateless/02880_indexHint__partition_id.sql diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 237a4cc703f..a2f2c1e0aac 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1346,10 +1346,7 @@ static void buildIndexes( } /// TODO Support row_policy_filter and additional_filters - if (settings.allow_experimental_analyzer) - indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context); - else - indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, query_info.query, context); + indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context); indexes->use_skip_indexes = settings.use_skip_indexes; bool final = query_info.isFinal(); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index d7c6c370c18..3c2b09b6f3b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -815,9 +815,6 @@ std::optional> MergeTreeDataSelectExecutor::filterPar ASTPtr expression_ast; auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */); - if (virtual_columns_block.rows() == 0) - return {}; - // Generate valid expressions for filtering VirtualColumnUtils::prepareFilterBlockWithQuery(query, context, virtual_columns_block, expression_ast); diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 652ab0b99de..4419190e12c 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -54,7 +54,6 @@ 01710_projection_additional_filters 01721_join_implicit_cast_long 01739_index_hint -02880_indexHint__partition_id 01747_join_view_filter_dictionary 01748_partition_id_pruning 01756_optimize_skip_unused_shards_rewrite_in diff --git a/tests/queries/0_stateless/02880_indexHint__partition_id.reference b/tests/queries/0_stateless/02880_indexHint__partition_id.reference deleted file mode 100644 index 365e7b676c7..00000000000 --- a/tests/queries/0_stateless/02880_indexHint__partition_id.reference +++ /dev/null @@ -1,9 +0,0 @@ --- { echoOn } -select * from data prewhere indexHint(_partition_id = '1'); -1 -select count() from data prewhere indexHint(_partition_id = '1'); -1 -select * from data where indexHint(_partition_id = '1'); -1 -select count() from data where indexHint(_partition_id = '1'); -1 diff --git a/tests/queries/0_stateless/02880_indexHint__partition_id.sql b/tests/queries/0_stateless/02880_indexHint__partition_id.sql deleted file mode 100644 index d15b3f4ccea..00000000000 --- a/tests/queries/0_stateless/02880_indexHint__partition_id.sql +++ /dev/null @@ -1,9 +0,0 @@ -drop table if exists data; -create table data (part Int) engine=MergeTree() order by tuple() partition by part; -insert into data values (1)(2); - --- { echoOn } -select * from data prewhere indexHint(_partition_id = '1'); -select count() from data prewhere indexHint(_partition_id = '1'); -select * from data where indexHint(_partition_id = '1'); -select count() from data where indexHint(_partition_id = '1'); From 7271cfd18704e2ab89ec9d8f3528e67794b6c7b0 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 19 Sep 2023 21:41:02 +0000 Subject: [PATCH 44/45] 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 696ab745aa44da168f98d7c024f325fc08c3be34 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 20 Sep 2023 07:51:27 +0200 Subject: [PATCH 45/45] 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`.