From 52dfaa54bf4cbba46ed45c6b3ec85f14259ee91c Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 8 Sep 2023 08:02:30 +0000 Subject: [PATCH 01/69] pass http retry timeout as ms Signed-off-by: Duc Canh Le --- src/IO/HTTPCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index add3e96c2c1..ef03acdde5d 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -321,7 +321,7 @@ namespace /// To avoid such a deadlock we unlock `lock` before entering `pool_ptr->second->get`. lock.unlock(); - auto retry_timeout = timeouts.connection_timeout.totalMicroseconds(); + auto retry_timeout = timeouts.connection_timeout.totalMilliseconds(); auto session = pool_ptr->second->get(retry_timeout); setTimeouts(*session, timeouts); From 2293923f66e1b2d0ccda674c2102b102a6c18944 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 11 Sep 2023 21:46:46 +0000 Subject: [PATCH 02/69] Disable parallel replicas on shards with not enough nodes --- src/Interpreters/Context.cpp | 14 ++++--- src/Interpreters/Context.h | 1 + src/Processors/QueryPlan/ReadFromRemote.cpp | 41 ++++++++++++++----- ...el_replicas_cluster_all_replicas.reference | 2 + ...parallel_replicas_cluster_all_replicas.sql | 13 ++++++ 5 files changed, 55 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.reference create mode 100644 tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 306db8ff491..ada77d821b0 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4624,18 +4624,20 @@ Context::ParallelReplicasMode Context::getParallelReplicasMode() const return SAMPLE_KEY; } -bool Context::canUseParallelReplicasOnInitiator() const +bool Context::canUseParallelReplicas() const { const auto & settings_ref = getSettingsRef(); - return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS && settings_ref.max_parallel_replicas > 1 - && !getClientInfo().collaborate_with_initiator; + return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS && settings_ref.max_parallel_replicas > 1; +} + +bool Context::canUseParallelReplicasOnInitiator() const +{ + 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 0eeea5b4fbd..1500c2f903c 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1170,6 +1170,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; diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 78da19f48a0..5f156bc624c 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -236,24 +236,45 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact scalars["_shard_num"] = Block{{DataTypeUInt32().createColumnConst(1, shard.shard_info.shard_num), std::make_shared(), "_shard_num"}}; - if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS) + ContextPtr execution_context = context; + if (context->canUseParallelReplicas()) { - if (context->getSettingsRef().cluster_for_parallel_replicas.changed) + if (shard.shard_info.getAllNodeCount() > 1) { - const String cluster_for_parallel_replicas = context->getSettingsRef().cluster_for_parallel_replicas; - if (cluster_for_parallel_replicas != cluster_name) - LOG_INFO(log, "cluster_for_parallel_replicas has been set for the query but has no effect: {}. Distributed table cluster is used: {}", - cluster_for_parallel_replicas, cluster_name); - } + if (context->getSettingsRef().cluster_for_parallel_replicas.changed) + { + const String cluster_for_parallel_replicas = context->getSettingsRef().cluster_for_parallel_replicas; + if (cluster_for_parallel_replicas != cluster_name) + LOG_INFO( + log, + "cluster_for_parallel_replicas has been set for the query but has no effect: {}. Distributed table cluster is " + "used: {}", + cluster_for_parallel_replicas, + cluster_name); + } - LOG_TRACE(&Poco::Logger::get("ReadFromRemote"), "Setting `cluster_for_parallel_replicas` to {}", cluster_name); - context->setSetting("cluster_for_parallel_replicas", cluster_name); + LOG_TRACE(&Poco::Logger::get("ReadFromRemote"), "Setting `cluster_for_parallel_replicas` to {}", cluster_name); + context->setSetting("cluster_for_parallel_replicas", cluster_name); + } + else + { + ContextMutablePtr tmp = Context::createCopy(context); + tmp->setSetting("allow_experimental_parallel_reading_from_replicas", Field{0}); + execution_context = tmp; + + LOG_TRACE( + &Poco::Logger::get("ReadFromRemote"), + "Parallel reading from replicas is disabled for shard. Not enough nodes: shard={} cluster={} nodes={}", + shard.shard_info.shard_num, + cluster_name, + shard.shard_info.getAllNodeCount()); + } } std::shared_ptr remote_query_executor; remote_query_executor = std::make_shared( - shard.shard_info.pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage); + shard.shard_info.pool, query_string, output_stream->header, execution_context, throttler, scalars, external_tables, stage); remote_query_executor->setLogger(log); diff --git a/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.reference b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.reference new file mode 100644 index 00000000000..79ebceba739 --- /dev/null +++ b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.reference @@ -0,0 +1,2 @@ +60 +1 diff --git a/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql new file mode 100644 index 00000000000..82a17e9bd01 --- /dev/null +++ b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS tt; +CREATE TABLE tt (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); +INSERT INTO tt SELECT * FROM numbers(10); + +SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, use_hedged_requests=0, parallel_replicas_for_non_replicated_merge_tree=1; +SELECT count() FROM clusterAllReplicas('test_cluster_two_shard_three_replicas_localhost', currentDatabase(), tt); + +SYSTEM FLUSH LOGS; + +SET allow_experimental_parallel_reading_from_replicas=0; +SELECT count() > 0 FROM system.text_log WHERE event_time >= now() - INTERVAL 2 MINUTE AND message LIKE '%Parallel reading from replicas is disabled for shard. Not enough nodes%'; + +DROP TABLE tt; From 1287f68745952aa84f19b58d24596b8d1c95f8b0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 12 Sep 2023 12:52:29 +0000 Subject: [PATCH 03/69] Handle clusterAllReplicas/remote cases to avoid unnecessary logging --- .../ClusterProxy/executeQuery.cpp | 37 ++++++++++++++++--- src/Interpreters/ClusterProxy/executeQuery.h | 2 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 2 +- src/Storages/getStructureOfRemoteTable.cpp | 4 +- ...parallel_replicas_cluster_all_replicas.sql | 6 ++- 5 files changed, 40 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 34be2636917..372376afc02 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -34,7 +34,7 @@ namespace ErrorCodes namespace ClusterProxy { -ContextMutablePtr updateSettingsForCluster(bool interserver_mode, +ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, const StorageID & main_table, @@ -47,6 +47,7 @@ ContextMutablePtr updateSettingsForCluster(bool interserver_mode, /// If "secret" (in remote_servers) is not in use, /// user on the shard is not the same as the user on the initiator, /// hence per-user limits should not be applied. + const bool interserver_mode = !cluster.getSecret().empty(); if (!interserver_mode) { /// Does not matter on remote servers, because queries are sent under different user. @@ -123,6 +124,22 @@ ContextMutablePtr updateSettingsForCluster(bool interserver_mode, new_settings.additional_table_filters.value.push_back(std::move(tuple)); } + /// disable parallel replicas if cluster contains only shards with 1 replica + if (context->canUseParallelReplicas()) + { + bool disable_parallel_replicas = false; + for (const auto & shard : cluster.getShardsInfo()) + { + if (shard.getAllNodeCount() <= 1) + { + disable_parallel_replicas = true; + break; + } + } + if (disable_parallel_replicas) + new_settings.allow_experimental_parallel_reading_from_replicas = false; + } + auto new_context = Context::createCopy(context); new_context->setSettings(new_settings); return new_context; @@ -174,12 +191,22 @@ void executeQuery( std::vector plans; SelectStreamFactory::Shards remote_shards; - auto new_context = updateSettingsForCluster(!not_optimized_cluster->getSecret().empty(), context, settings, - main_table, query_info.additional_filter_ast, log); + auto cluster = query_info.getCluster(); + auto new_context + = updateSettingsForCluster(*cluster, context, settings, main_table, query_info.additional_filter_ast, log); + if (new_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value + != context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value) + { + LOG_TRACE( + log, + "Parallel reading from replicas is disabled for cluster. There are no shards with more then 1 replica: cluster={}", + cluster->getName()); + } + new_context->increaseDistributedDepth(); - size_t shards = query_info.getCluster()->getShardCount(); - for (const auto & shard_info : query_info.getCluster()->getShardsInfo()) + 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) diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index b663dffb7fa..032458bea60 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -37,7 +37,7 @@ class SelectStreamFactory; /// - optimize_skip_unused_shards_nesting /// /// @return new Context with adjusted settings -ContextMutablePtr updateSettingsForCluster(bool interserver_mode, +ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, const StorageID & main_table, diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 5f156bc624c..884bc85aaaf 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -278,7 +278,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact remote_query_executor->setLogger(log); - if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS) + if (execution_context->canUseParallelReplicas()) { // when doing parallel reading from replicas (ParallelReplicasMode::READ_TASKS) on a shard: // establish a connection to a replica on the shard, the replica will instantiate coordinator to manage parallel reading from replicas on the shard. diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index ebd02f424fa..32266f20923 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -57,7 +57,7 @@ ColumnsDescription getStructureOfRemoteTableInShard( } ColumnsDescription res; - auto new_context = ClusterProxy::updateSettingsForCluster(!cluster.getSecret().empty(), context, context->getSettingsRef(), table_id); + auto new_context = ClusterProxy::updateSettingsForCluster(cluster, context, context->getSettingsRef(), table_id); /// Ignore limit for result number of rows (that could be set during handling CSE/CTE), /// since this is a service query and should not lead to query failure. @@ -176,7 +176,7 @@ ColumnsDescriptionByShardNum getExtendedObjectsOfRemoteTables( const auto & shards_info = cluster.getShardsInfo(); auto query = "DESC TABLE " + remote_table_id.getFullTableName(); - auto new_context = ClusterProxy::updateSettingsForCluster(!cluster.getSecret().empty(), context, context->getSettingsRef(), remote_table_id); + auto new_context = ClusterProxy::updateSettingsForCluster(cluster, context, context->getSettingsRef(), remote_table_id); new_context->setSetting("describe_extend_object_types", true); /// Expect only needed columns from the result of DESC TABLE. diff --git a/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql index 82a17e9bd01..c1cbc404aa6 100644 --- a/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql +++ b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql @@ -3,11 +3,13 @@ CREATE TABLE tt (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); INSERT INTO tt SELECT * FROM numbers(10); SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, use_hedged_requests=0, parallel_replicas_for_non_replicated_merge_tree=1; -SELECT count() FROM clusterAllReplicas('test_cluster_two_shard_three_replicas_localhost', currentDatabase(), tt); +SELECT count() FROM clusterAllReplicas('test_cluster_two_shard_three_replicas_localhost', currentDatabase(), tt) settings log_comment='02875_190aed82-2423-413b-ad4c-24dcca50f65b'; SYSTEM FLUSH LOGS; SET allow_experimental_parallel_reading_from_replicas=0; -SELECT count() > 0 FROM system.text_log WHERE event_time >= now() - INTERVAL 2 MINUTE AND message LIKE '%Parallel reading from replicas is disabled for shard. Not enough nodes%'; +SELECT count() > 0 FROM system.text_log +WHERE query_id in (select query_id from system.query_log where log_comment = '02875_190aed82-2423-413b-ad4c-24dcca50f65b') + AND message LIKE '%Parallel reading from replicas is disabled for cluster%'; DROP TABLE tt; From f427ca771e60953209918eb3fe009c54f50d8444 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 14 Sep 2023 12:41:43 +0200 Subject: [PATCH 04/69] Fix optimize read in order for views. --- src/Functions/materialize.h | 9 ++++++++ ...optimize_read_in_order_from_view.reference | 21 +++++++++++++++++++ ...02877_optimize_read_in_order_from_view.sql | 21 +++++++++++++++++++ 3 files changed, 51 insertions(+) create mode 100644 tests/queries/0_stateless/02877_optimize_read_in_order_from_view.reference create mode 100644 tests/queries/0_stateless/02877_optimize_read_in_order_from_view.sql diff --git a/src/Functions/materialize.h b/src/Functions/materialize.h index aab4e5bdbdf..73bfdec48ab 100644 --- a/src/Functions/materialize.h +++ b/src/Functions/materialize.h @@ -52,6 +52,15 @@ public: { return arguments[0].column->convertToFullColumnIfConst(); } + + bool hasInformationAboutMonotonicity() const override { return true; } + + Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override + { + /// Depending on the argument the function materialize() is either a constant or works as identity(). + /// In both cases this function is monotonic and non-decreasing. + return {.is_monotonic = true, .is_always_monotonic = true}; + } }; } diff --git a/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.reference b/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.reference new file mode 100644 index 00000000000..05893173f2b --- /dev/null +++ b/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.reference @@ -0,0 +1,21 @@ +300 +299 +298 +297 +296 +295 +294 +293 +292 +291 +290 +289 +288 +287 +286 +285 +284 +283 +282 +281 +read_rows=ok diff --git a/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.sql b/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.sql new file mode 100644 index 00000000000..3397681d0d9 --- /dev/null +++ b/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.sql @@ -0,0 +1,21 @@ +SET optimize_read_in_order=1; + +DROP TABLE IF EXISTS view1; +DROP TABLE IF EXISTS table1; + +CREATE TABLE table1 (number UInt64) ENGINE=MergeTree ORDER BY number SETTINGS index_granularity=1; +INSERT INTO table1 SELECT number FROM numbers(1, 300); + +CREATE VIEW view1 AS SELECT number FROM table1; + +-- The following SELECT is expected to read 20 rows. In fact it may decide to read more than 20 rows, but not too many anyway. +-- So we'll check that the number of read rows is less than 40. + +SELECT /* test 02877, query 1 */ * FROM (SELECT * FROM view1) ORDER BY number DESC LIMIT 20 SETTINGS log_queries=1; + +SYSTEM FLUSH LOGS; + +SELECT concat('read_rows=', if(read_rows<40, 'ok', toString(read_rows))) FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE '%test 02877, query 1%' AND type='QueryFinish'; + +DROP TABLE view1; +DROP TABLE table1; From 52de64c2bd019ae9c06a6a3aa1c770f4dcd0946c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 14 Sep 2023 18:30:12 +0200 Subject: [PATCH 05/69] remove unneeded code --- src/Storages/StorageReplicatedMergeTree.cpp | 22 ++++++++------------- 1 file changed, 8 insertions(+), 14 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7c7e6dbd42c..e56055a717a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9033,7 +9033,7 @@ namespace /// But sometimes we need an opposite. When we deleting all_0_0_0_1 it can be non replicated to other replicas, so we are the only owner of this part. /// In this case when we will drop all_0_0_0_1 we will drop blobs for all_0_0_0. But it will lead to dataloss. For such case we need to check that other replicas /// still need parent part. -std::pair> getParentLockedBlobs(const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const MergeTreePartInfo & part_info, MergeTreeDataFormatVersion format_version, Poco::Logger * log) +std::pair getParentLockedBlobs(const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const MergeTreePartInfo & part_info, MergeTreeDataFormatVersion format_version, Poco::Logger * log) { NameSet files_not_to_remove; @@ -9078,8 +9078,9 @@ std::pair> getParentLockedBlobs(const ZooKeeperWith zookeeper_ptr->tryGet(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, files_not_to_remove_str, nullptr, nullptr, &code); if (code != Coordination::Error::ZOK) { - LOG_TRACE(log, "Cannot get parent files from ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), code); - return {true, std::nullopt}; + LOG_INFO(log, "Cannot get parent files from ZooKeeper on path ({}), error {}, assuming the parent was removed concurrently", + (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), code); + continue; } if (!files_not_to_remove_str.empty()) @@ -9093,8 +9094,9 @@ std::pair> getParentLockedBlobs(const ZooKeeperWith code = zookeeper_ptr->tryGetChildren(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, children); if (code != Coordination::Error::ZOK) { - LOG_TRACE(log, "Cannot get parent locks in ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), errorMessage(code)); - return {true, std::nullopt}; + LOG_INFO(log, "Cannot get parent locks in ZooKeeper on path ({}), error {}, assuming the parent was removed concurrently", + (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), errorMessage(code)); + continue; } if (children.size() > 1 || (children.size() == 1 && children[0] != ZeroCopyLock::ZERO_COPY_LOCK_NAME)) @@ -9150,15 +9152,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( auto [has_parent, parent_not_to_remove] = getParentLockedBlobs( zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_info, data_format_version, logger); - // parent_not_to_remove == std::nullopt means that we were unable to retrieve parts set - if (has_parent && parent_not_to_remove == std::nullopt) - { - LOG_TRACE(logger, "Failed to get mutation parent on {} for part {}, refusing to remove blobs", zookeeper_part_replica_node, part_name); - return {false, {}}; - } - - files_not_to_remove.insert(parent_not_to_remove->begin(), parent_not_to_remove->end()); - + files_not_to_remove.insert(parent_not_to_remove.begin(), parent_not_to_remove.end()); LOG_TRACE(logger, "Remove zookeeper lock {} for part {}", zookeeper_part_replica_node, part_name); From c422a8f0dc14269041aba360ea0ac645e2a635e1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 13 Sep 2023 13:41:02 +0000 Subject: [PATCH 06/69] Cosmetics --- docs/en/sql-reference/data-types/array.md | 2 +- .../MergeTree/MergeTreeIndexAnnoy.cpp | 26 +++++++++---------- .../MergeTree/MergeTreeIndexUSearch.cpp | 24 ++++++++--------- 3 files changed, 26 insertions(+), 26 deletions(-) diff --git a/docs/en/sql-reference/data-types/array.md b/docs/en/sql-reference/data-types/array.md index 20ce7d2ed52..0ee7c8de93c 100644 --- a/docs/en/sql-reference/data-types/array.md +++ b/docs/en/sql-reference/data-types/array.md @@ -4,7 +4,7 @@ sidebar_position: 52 sidebar_label: Array(T) --- -# Array(t) +# Array(T) An array of `T`-type items, with the starting array index as 1. `T` can be any data type, including an array. diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 3ad4f81716e..15830513162 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -154,36 +154,36 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t if (const auto & column_array = typeid_cast(column_cut.get())) { - const auto & data = column_array->getData(); - const auto & array = typeid_cast(data).getData(); + const auto & column_array_data = column_array->getData(); + const auto & column_arary_data_float_data = typeid_cast(column_array_data).getData(); - if (array.empty()) + if (column_arary_data_float_data.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read); - const auto & offsets = column_array->getOffsets(); - const size_t num_rows = offsets.size(); + const auto & column_array_offsets = column_array->getOffsets(); + const size_t num_rows = column_array_offsets.size(); /// Check all sizes are the same - size_t size = offsets[0]; + size_t dimension = column_array_offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) - if (offsets[i + 1] - offsets[i] != size) + if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension) throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); if (!index) - index = std::make_shared>(size); + index = std::make_shared>(dimension); /// Add all rows of block - index->add_item(index->get_n_items(), array.data()); + index->add_item(index->get_n_items(), column_arary_data_float_data.data()); for (size_t current_row = 1; current_row < num_rows; ++current_row) - index->add_item(index->get_n_items(), &array[offsets[current_row - 1]]); + index->add_item(index->get_n_items(), &column_arary_data_float_data[column_array_offsets[current_row - 1]]); } else if (const auto & column_tuple = typeid_cast(column_cut.get())) { - const auto & columns = column_tuple->getColumns(); + const auto & column_tuple_columns = column_tuple->getColumns(); /// TODO check if calling index->add_item() directly on the block's tuples is faster than materializing everything - std::vector> data{column_tuple->size(), std::vector()}; - for (const auto & column : columns) + std::vector> data(column_tuple->size(), std::vector()); + for (const auto & column : column_tuple_columns) { const auto & pod_array = typeid_cast(column.get())->getData(); for (size_t i = 0; i < pod_array.size(); ++i) diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index 1ab85e6bbaf..de556eb7e07 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -173,23 +173,23 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t if (const auto & column_array = typeid_cast(column_cut.get())) { - const auto & data = column_array->getData(); - const auto & array = typeid_cast(data).getData(); + const auto & column_array_data = column_array->getData(); + const auto & column_array_data_float_data = typeid_cast(column_array_data).getData(); - if (array.empty()) + if (column_array_data_float_data.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read); - const auto & offsets = column_array->getOffsets(); - const size_t num_rows = offsets.size(); + const auto & column_array_offsets = column_array->getOffsets(); + const size_t num_rows = column_array_offsets.size(); /// Check all sizes are the same - size_t size = offsets[0]; + size_t dimension = column_array_offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) - if (offsets[i + 1] - offsets[i] != size) + if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension) throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); if (!index) - index = std::make_shared>(size, scalar_kind); + index = std::make_shared>(dimension, scalar_kind); /// Add all rows of block if (!index->reserve(unum::usearch::ceil2(index->size() + num_rows))) @@ -197,7 +197,7 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t for (size_t current_row = 0; current_row < num_rows; ++current_row) { - auto rc = index->add(static_cast(index->size()), &array[offsets[current_row - 1]]); + auto rc = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[current_row - 1]]); if (!rc) throw Exception(ErrorCodes::INCORRECT_DATA, rc.error.release()); @@ -208,9 +208,9 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t } else if (const auto & column_tuple = typeid_cast(column_cut.get())) { - const auto & columns = column_tuple->getColumns(); - std::vector> data{column_tuple->size(), std::vector()}; - for (const auto & column : columns) + const auto & column_tuple_columns = column_tuple->getColumns(); + std::vector> data(column_tuple->size(), std::vector()); + for (const auto & column : column_tuple_columns) { const auto & pod_array = typeid_cast(column.get())->getData(); for (size_t i = 0; i < pod_array.size(); ++i) From 945179be46bcc1b07741d180a0fdaa64396994ef Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 13 Sep 2023 14:23:09 +0000 Subject: [PATCH 07/69] Annoy: Fix LOGICAL_ERROR with default values #52258 --- .../table-engines/mergetree-family/annindexes.md | 8 +++++--- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 15 +++++++++++---- src/Storages/MergeTree/MergeTreeIndexUSearch.cpp | 16 ++++++++++++---- .../0_stateless/02354_annoy_index.reference | 1 + tests/queries/0_stateless/02354_annoy_index.sql | 12 ++++++++++++ .../0_stateless/02354_usearch_index.reference | 1 + .../queries/0_stateless/02354_usearch_index.sql | 14 ++++++++++++++ 7 files changed, 56 insertions(+), 11 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 8996133f667..d6ff7f23bb4 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -203,9 +203,10 @@ Parameter `NumTrees` is the number of trees which the algorithm creates (default more accurate search results but slower index creation / query times (approximately linearly) as well as larger index sizes. :::note -Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use -[CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 -CHECK length(vectors) = 256`. +Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays must have same length. To avoid +errors, you can use a [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints), for example, `CONSTRAINT +constraint_name_1 CHECK length(vectors) = 256`. Also, unspecified `Array` values in INSERT statements (i.e. default values) are not +supported. ::: Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. Larger @@ -223,6 +224,7 @@ SETTINGS annoy_index_search_k_nodes=100; The Annoy index currently does not work with per-table, non-default `index_granularity` settings (see [here](https://github.com/ClickHouse/ClickHouse/pull/51325#issuecomment-1605920475)). If necessary, the value must be changed in config.xml. ::: + ## USearch {#usearch} This type of ANN index is based on the [the USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 15830513162..f00f11359e1 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -157,18 +157,25 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t const auto & column_array_data = column_array->getData(); const auto & column_arary_data_float_data = typeid_cast(column_array_data).getData(); - if (column_arary_data_float_data.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read); - const auto & column_array_offsets = column_array->getOffsets(); const size_t num_rows = column_array_offsets.size(); + /// The index dimension is inferred from the inserted arrays (array cardinality). If no value was specified in the INSERT statement + /// for the annoy-indexed column (i.e. default value), we have a problem. Reject such values. + if (column_array_offsets.empty() || column_array_offsets[0] == 0) + /// (The if condition is a bit weird but I have seen either with default values) + throw Exception(ErrorCodes::INCORRECT_DATA, "Tried to insert {} rows into Annoy index but there were no values to insert. Likely, the INSERT used default values - these are not supported for Annoy.", rows_read); + /// Check all sizes are the same size_t dimension = column_array_offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension) throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + /// Also check that previously inserted blocks have the same size as this block + if (index && index->getDimensions() != dimension) + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + if (!index) index = std::make_shared>(dimension); @@ -363,7 +370,7 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) { throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Annoy indexes can only be created on columns of type Array(Float32) and Tuple(Float32)"); + "Annoy indexes can only be created on columns of type Array(Float32) and Tuple(Float32[, Float32[, ...]])"); }; DataTypePtr data_type = index.sample_block.getDataTypes()[0]; diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index de556eb7e07..9531b9188bf 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -176,18 +176,25 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t const auto & column_array_data = column_array->getData(); const auto & column_array_data_float_data = typeid_cast(column_array_data).getData(); - if (column_array_data_float_data.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read); - const auto & column_array_offsets = column_array->getOffsets(); const size_t num_rows = column_array_offsets.size(); + /// The index dimension is inferred from the inserted arrays (array cardinality). If no value was specified in the INSERT statement + /// for the usearch-indexed column (i.e. default value), we have a problem. Reject such values. + if (column_array_offsets.empty() || column_array_offsets[0] == 0) + /// (The if condition is a bit weird but I have seen either with default values) + throw Exception(ErrorCodes::INCORRECT_DATA, "Tried to insert {} rows into usearch index but there were no values to insert. Likely, the INSERT used default values - these are not supported for Annoy.", rows_read); + /// Check all sizes are the same size_t dimension = column_array_offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension) throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + /// Also check that previously inserted blocks have the same size as this block + if (index && index->getDimensions() != dimension) + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + if (!index) index = std::make_shared>(dimension, scalar_kind); @@ -413,7 +420,8 @@ void usearchIndexValidator(const IndexDescription & index, bool /* attach */) auto throw_unsupported_underlying_column_exception = []() { throw Exception( - ErrorCodes::ILLEGAL_COLUMN, "USearch indexes can only be created on columns of type Array(Float32) and Tuple(Float32)"); + ErrorCodes::ILLEGAL_COLUMN, + "USearch can only be created on columns of type Array(Float32) and Tuple(Float32[, Float32[, ...]])"); }; DataTypePtr data_type = index.sample_block.getDataTypes()[0]; diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference index a0ffb1e1f7f..81f2ff8aa59 100644 --- a/tests/queries/0_stateless/02354_annoy_index.reference +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -147,3 +147,4 @@ Expression (Projection) 9000 [9000,0,0,0] 1 (1,0,0,0) 9000 (9000,0,0,0) +--- Bugs --- diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index eab7a62c5f0..67ef64cc301 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -281,3 +281,15 @@ ORDER BY L2Distance(vector, (9000.0, 0.0, 0.0, 0.0)) LIMIT 1; DROP TABLE tab; + +SELECT '--- Bugs ---'; + +-- Arrays with default values are rejected, issue #52258 +CREATE TABLE tab (`uuid` String, `vector` Array(Float32), `version` UInt32, INDEX idx vector TYPE annoy()) ENGINE = MergeTree() ORDER BY (uuid); +INSERT INTO tab (uuid, version) VALUES ('1', 3); -- { serverError INCORRECT_DATA } +DROP TABLE tab; + +-- Tuples with default value work +CREATE TABLE tab (`uuid` String, `vector` Tuple(Float32, Float32), `version` UInt32, INDEX idx vector TYPE annoy()) ENGINE = MergeTree() ORDER BY (uuid); +INSERT INTO tab (uuid, version) VALUES ('1', 3); -- works fine +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_usearch_index.reference b/tests/queries/0_stateless/02354_usearch_index.reference index 893a092a386..c2791e99a54 100644 --- a/tests/queries/0_stateless/02354_usearch_index.reference +++ b/tests/queries/0_stateless/02354_usearch_index.reference @@ -150,3 +150,4 @@ Expression (Projection) 1 [0,0,10] 2 [0,0,10.5] 3 [0,0,9.5] +--- Bugs --- diff --git a/tests/queries/0_stateless/02354_usearch_index.sql b/tests/queries/0_stateless/02354_usearch_index.sql index e534c91b615..fc2954d6c5d 100644 --- a/tests/queries/0_stateless/02354_usearch_index.sql +++ b/tests/queries/0_stateless/02354_usearch_index.sql @@ -274,3 +274,17 @@ SELECT * FROM tab WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < 1.0 LIMIT 3; + +DROP TABLE tab; + +SELECT '--- Bugs ---'; + +-- Arrays with default values are rejected, issue #52258 +CREATE TABLE tab (`uuid` String, `vector` Array(Float32), `version` UInt32, INDEX idx vector TYPE usearch()) ENGINE = MergeTree() ORDER BY (uuid); +INSERT INTO tab (uuid, version) VALUES ('1', 3); -- { serverError INCORRECT_DATA } +DROP TABLE tab; + +-- Tuples with default value work +CREATE TABLE tab (`uuid` String, `vector` Tuple(Float32, Float32), `version` UInt32, INDEX idx vector TYPE usearch()) ENGINE = MergeTree() ORDER BY (uuid); +INSERT INTO tab (uuid, version) VALUES ('1', 3); -- works fine +DROP TABLE tab; From 32cad222bf4875075d7a7e41c6095b8ae11109a3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 12 Sep 2023 17:10:57 +0300 Subject: [PATCH 08/69] KeyCondition IN function support different types --- src/Storages/MergeTree/KeyCondition.cpp | 48 ++++++++-- src/Storages/MergeTree/RPNBuilder.cpp | 58 ------------ src/Storages/MergeTree/RPNBuilder.h | 5 -- ...ndex_in_function_different_types.reference | 88 +++++++++++++++++++ ..._key_index_in_function_different_types.sql | 24 +++++ 5 files changed, 155 insertions(+), 68 deletions(-) create mode 100644 tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference create mode 100644 tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 42731bac19b..bea0111fdf3 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1253,10 +1253,21 @@ bool KeyCondition::tryPrepareSetIndex( const auto right_arg = func.getArgumentAt(1); - auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); + auto future_set = right_arg.tryGetPreparedSet(); if (!future_set) return false; + const auto & set_types = future_set->getTypes(); + size_t set_types_size = set_types.size(); + size_t indexes_mapping_size = indexes_mapping.size(); + + if (set_types_size != indexes_mapping_size) + return false; + + for (auto & index_mapping : indexes_mapping) + if (index_mapping.tuple_index >= set_types_size) + return false; + auto prepared_set = future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); if (!prepared_set) return false; @@ -1265,11 +1276,38 @@ bool KeyCondition::tryPrepareSetIndex( if (!prepared_set->hasExplicitSetElements()) return false; - prepared_set->checkColumnsNumber(left_args_count); - for (size_t i = 0; i < indexes_mapping.size(); ++i) - prepared_set->checkTypesEqual(indexes_mapping[i].tuple_index, data_types[i]); + /** Try to convert set columns to primary key columns. + * Example: SELECT id FROM test_table WHERE id IN (SELECT 1); + * In this example table `id` column has type UInt64, Set column has type UInt8. To use index + * we need to convert set column to primary key column. + */ + const auto & set_elements = prepared_set->getSetElements(); + size_t set_elements_size = set_elements.size(); + assert(set_types_size == set_elements_size); - out.set_index = std::make_shared(prepared_set->getSetElements(), std::move(indexes_mapping)); + Columns set_columns; + set_columns.reserve(set_elements_size); + + for (size_t i = 0; i < indexes_mapping_size; ++i) + { + size_t set_element_index = indexes_mapping[i].tuple_index; + const auto & set_element = set_elements[set_element_index]; + const auto & set_element_type = set_types[set_element_index]; + + ColumnPtr set_column; + try + { + set_column = castColumnAccurate({set_element, set_element_type, {}}, data_types[i]); + } + catch (...) + { + return false; + } + + set_columns.push_back(set_column); + } + + out.set_index = std::make_shared(set_columns, std::move(indexes_mapping)); return true; } diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index a0c96c13d59..29bcd8b87f1 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -351,64 +351,6 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) return nullptr; } -FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( - const std::vector & indexes_mapping, - const DataTypes & data_types) const -{ - const auto & prepared_sets = getTreeContext().getPreparedSets(); - - /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information - /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets - /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check - /// that the types it was prepared with are compatible with the types of the primary key. - auto types_match = [&indexes_mapping, &data_types](const DataTypes & set_types) - { - assert(indexes_mapping.size() == data_types.size()); - - for (size_t i = 0; i < indexes_mapping.size(); ++i) - { - if (indexes_mapping[i].tuple_index >= set_types.size()) - return false; - - auto lhs = removeNullable(recursiveRemoveLowCardinality(data_types[i])); - auto rhs = removeNullable(recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index])); - - if (!lhs->equals(*rhs)) - return false; - } - - return true; - }; - - if (prepared_sets && ast_node) - { - if (ast_node->as() || ast_node->as()) - return prepared_sets->findSubquery(ast_node->getTreeHash()); - - auto tree_hash = ast_node->getTreeHash(); - const auto & sets = prepared_sets->getSetsFromTuple(); - auto it = sets.find(tree_hash); - if (it == sets.end()) - return nullptr; - - for (const auto & future_set : it->second) - if (types_match(future_set->getTypes())) - return future_set; - } - else - { - const auto * node_without_alias = getNodeWithoutAlias(dag_node); - if (node_without_alias->column) - { - auto future_set = tryGetSetFromDAGNode(node_without_alias); - if (types_match(future_set->getTypes())) - return future_set; - } - } - - return nullptr; -} - RPNBuilderFunctionTreeNode RPNBuilderTreeNode::toFunctionNode() const { if (!isFunction()) diff --git a/src/Storages/MergeTree/RPNBuilder.h b/src/Storages/MergeTree/RPNBuilder.h index 9eeb6deefd5..f14f241cac8 100644 --- a/src/Storages/MergeTree/RPNBuilder.h +++ b/src/Storages/MergeTree/RPNBuilder.h @@ -116,11 +116,6 @@ public: /// Try get prepared set from node that match data types FutureSetPtr tryGetPreparedSet(const DataTypes & data_types) const; - /// Try get prepared set from node that match indexes mapping and data types - FutureSetPtr tryGetPreparedSet( - const std::vector & indexes_mapping, - const DataTypes & data_types) const; - /** Convert node to function node. * Node must be function before calling these method, otherwise exception is thrown. */ diff --git a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference new file mode 100644 index 00000000000..f34aad737d4 --- /dev/null +++ b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference @@ -0,0 +1,88 @@ +CreatingSets (Create sets before main query execution) + Expression ((Projection + Before ORDER BY)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 1-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Projection + Before ORDER BY)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 1-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Projection + Before ORDER BY)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 5-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Projection + Before ORDER BY)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 5-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Project names + Projection)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 1-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Project names + Projection)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 1-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Project names + Projection)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 5-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets (Create sets before main query execution) + Expression ((Project names + Projection)) + ReadFromMergeTree (default.test_table) + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 5-element set)) + Parts: 1/1 + Granules: 1/1 diff --git a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql new file mode 100644 index 00000000000..648a8041b71 --- /dev/null +++ b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql @@ -0,0 +1,24 @@ +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value UInt64 +) ENGINE=MergeTree ORDER BY (id, value); + +INSERT INTO test_table SELECT number, number FROM numbers(10); + +SET allow_experimental_analyzer = 0; + +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT 5); +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT '5'); +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toUInt8(number) FROM numbers(5)); +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toString(number) FROM numbers(5)); + +SET allow_experimental_analyzer = 1; + +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT 5); +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT '5'); +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toUInt8(number) FROM numbers(5)); +EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toString(number) FROM numbers(5)); + +DROP TABLE test_table; From 60a63212c876995c7d7b458e767f87f7a6847c76 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 12 Sep 2023 21:26:53 +0300 Subject: [PATCH 09/69] Fixed tests --- src/Interpreters/PreparedSets.cpp | 11 +++++++---- src/Interpreters/PreparedSets.h | 8 ++++---- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index e0551dff2ad..955d8892284 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -48,7 +48,7 @@ static bool equals(const DataTypes & lhs, const DataTypes & rhs) FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} SetPtr FutureSetFromStorage::get() const { return set; } -const DataTypes & FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); } +DataTypes FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); } SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) { @@ -73,7 +73,7 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) set->finishInsert(); } -const DataTypes & FutureSetFromTuple::getTypes() const { return set->getElementsTypes(); } +DataTypes FutureSetFromTuple::getTypes() const { return set->getElementsTypes(); } SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) { @@ -138,7 +138,7 @@ void FutureSetFromSubquery::setQueryPlan(std::unique_ptr source_) set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); } -const DataTypes & FutureSetFromSubquery::getTypes() const +DataTypes FutureSetFromSubquery::getTypes() const { return set_and_key->set->getElementsTypes(); } @@ -183,7 +183,10 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) { auto set = external_table_set->buildOrderedSetInplace(context); if (set) - return set_and_key->set = set; + { + set_and_key->set = set; + return set_and_key->set; + } } auto plan = build(context); diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index b953b8470e1..e237789c63c 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -47,7 +47,7 @@ public: /// Returns set if set is ready (created and filled) or nullptr if not. virtual SetPtr get() const = 0; /// Returns set->getElementsTypes(), even if set is not created yet. - virtual const DataTypes & getTypes() const = 0; + virtual DataTypes getTypes() const = 0; /// If possible, return set with stored elements useful for PK analysis. virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; }; @@ -62,7 +62,7 @@ public: FutureSetFromStorage(SetPtr set_); SetPtr get() const override; - const DataTypes & getTypes() const override; + DataTypes getTypes() const override; SetPtr buildOrderedSetInplace(const ContextPtr &) override; private: @@ -79,7 +79,7 @@ public: SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - const DataTypes & getTypes() const override; + DataTypes getTypes() const override; private: SetPtr set; @@ -105,7 +105,7 @@ public: const Settings & settings); SetPtr get() const override; - const DataTypes & getTypes() const override; + DataTypes getTypes() const override; SetPtr buildOrderedSetInplace(const ContextPtr & context) override; std::unique_ptr build(const ContextPtr & context); From 7a7697059c6b5fb8fb0925429f2d3cf0cc31f8c9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 13 Sep 2023 17:44:38 +0300 Subject: [PATCH 10/69] Fixed tests --- src/Storages/MergeTree/KeyCondition.cpp | 20 +++++--------------- 1 file changed, 5 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index bea0111fdf3..53b7e170ea5 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1261,9 +1261,6 @@ bool KeyCondition::tryPrepareSetIndex( size_t set_types_size = set_types.size(); size_t indexes_mapping_size = indexes_mapping.size(); - if (set_types_size != indexes_mapping_size) - return false; - for (auto & index_mapping : indexes_mapping) if (index_mapping.tuple_index >= set_types_size) return false; @@ -1281,33 +1278,26 @@ bool KeyCondition::tryPrepareSetIndex( * In this example table `id` column has type UInt64, Set column has type UInt8. To use index * we need to convert set column to primary key column. */ - const auto & set_elements = prepared_set->getSetElements(); - size_t set_elements_size = set_elements.size(); - assert(set_types_size == set_elements_size); - - Columns set_columns; - set_columns.reserve(set_elements_size); + auto set_elements = prepared_set->getSetElements(); + assert(set_types_size == set_elements.size()); for (size_t i = 0; i < indexes_mapping_size; ++i) { size_t set_element_index = indexes_mapping[i].tuple_index; - const auto & set_element = set_elements[set_element_index]; const auto & set_element_type = set_types[set_element_index]; + auto & set_element = set_elements[set_element_index]; - ColumnPtr set_column; try { - set_column = castColumnAccurate({set_element, set_element_type, {}}, data_types[i]); + set_element = castColumnAccurate({set_element, set_element_type, {}}, data_types[i]); } catch (...) { return false; } - - set_columns.push_back(set_column); } - out.set_index = std::make_shared(set_columns, std::move(indexes_mapping)); + out.set_index = std::make_shared(set_elements, std::move(indexes_mapping)); return true; } From 29e4352c17d49e317914fae7e7bd01cceed62d1b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Sep 2023 19:54:39 +0300 Subject: [PATCH 11/69] Updated implementation --- src/Storages/MergeTree/KeyCondition.cpp | 46 +++++++++++++++++++------ 1 file changed, 36 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 53b7e170ea5..47521b9887b 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1278,26 +1278,52 @@ bool KeyCondition::tryPrepareSetIndex( * In this example table `id` column has type UInt64, Set column has type UInt8. To use index * we need to convert set column to primary key column. */ - auto set_elements = prepared_set->getSetElements(); - assert(set_types_size == set_elements.size()); + auto set_columns = prepared_set->getSetElements(); + assert(set_types_size == set_columns.size()); - for (size_t i = 0; i < indexes_mapping_size; ++i) + for (size_t indexes_mapping_index = 0; indexes_mapping_index < indexes_mapping_size; ++indexes_mapping_index) { - size_t set_element_index = indexes_mapping[i].tuple_index; + size_t set_element_index = indexes_mapping[indexes_mapping_index].tuple_index; const auto & set_element_type = set_types[set_element_index]; - auto & set_element = set_elements[set_element_index]; + auto & set_column = set_columns[set_element_index]; - try + bool is_set_column_nullable = set_element_type->isNullable(); + bool is_set_column_low_cardinality_nullable = set_element_type->isLowCardinalityNullable(); + + const NullMap * set_column_null_map = nullptr; + + if (is_set_column_nullable || is_set_column_low_cardinality_nullable) { - set_element = castColumnAccurate({set_element, set_element_type, {}}, data_types[i]); + if (is_set_column_low_cardinality_nullable) + set_column = set_column->convertToFullColumnIfLowCardinality(); + + set_column_null_map = &assert_cast(*set_column).getNullMapData(); } - catch (...) + + auto nullable_set_column = castColumnAccurateOrNull({set_column, set_element_type, {}}, data_types[indexes_mapping_index]); + const auto & nullable_set_column_typed = assert_cast(*nullable_set_column); + const auto & nullable_set_column_null_map = nullable_set_column_typed.getNullMapData(); + size_t nullable_set_column_null_map_size = nullable_set_column_null_map.size(); + + IColumn::Filter filter(nullable_set_column_null_map_size); + + if (set_column_null_map) { - return false; + for (size_t i = 0; i < nullable_set_column_null_map_size; ++i) + filter[i] = (*set_column_null_map)[i] || !nullable_set_column_null_map[i]; + + set_column = nullable_set_column_typed.filter(filter, 0); + } + else + { + for (size_t i = 0; i < nullable_set_column_null_map_size; ++i) + filter[i] = !nullable_set_column_null_map[i]; + + set_column = nullable_set_column_typed.getNestedColumn().filter(filter, 0); } } - out.set_index = std::make_shared(set_elements, std::move(indexes_mapping)); + out.set_index = std::make_shared(set_columns, std::move(indexes_mapping)); return true; } From 8ef910e1fa0c4d8d6053b111efab5b4aecaaffb2 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 15 Sep 2023 22:34:09 +0200 Subject: [PATCH 12/69] Updated to check if cleanup is enabled before adding to cleanedup_rows_count --- .../Algorithms/ReplacingSortedAlgorithm.cpp | 4 ++-- ...lacing_merge_tree_vertical_merge.reference | 3 +++ ...77_replacing_merge_tree_vertical_merge.sql | 21 ++++++++++++++++++- 3 files changed, 25 insertions(+), 3 deletions(-) diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp index 4d2443b1e46..18f144bee3d 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp @@ -83,7 +83,7 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() uint8_t value = assert_cast(*(*selected_row.all_columns)[is_deleted_column_number]).getData()[selected_row.row_num]; if (!cleanup || !value) insertRow(); - else if (cleanedup_rows_count != nullptr) + else if (cleanup && cleanedup_rows_count != nullptr) *cleanedup_rows_count += current_row_sources.size(); } else @@ -141,7 +141,7 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() uint8_t value = assert_cast(*(*selected_row.all_columns)[is_deleted_column_number]).getData()[selected_row.row_num]; if (!cleanup || !value) insertRow(); - else if (cleanedup_rows_count != nullptr) + else if (cleanup && cleanedup_rows_count != nullptr) *cleanedup_rows_count += current_row_sources.size(); } else diff --git a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference index e0f8c3bae3f..6bac6173183 100644 --- a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference +++ b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference @@ -2,3 +2,6 @@ 2018-01-01 1 1 2018-01-01 2 2 2018-01-01 2 2 +== (Replicas) Test optimize == +d2 1 0 +d4 1 0 diff --git a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql index 8c51a6f34da..931297fdd3b 100644 --- a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql +++ b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql @@ -1,10 +1,29 @@ set optimize_on_insert = 0; drop table if exists tab_00577; -create table tab_00577 (date Date, version UInt64, val UInt64) engine = ReplacingMergeTree(version) partition by date order by date settings enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 0; +create table tab_00577 (date Date, version UInt64, val UInt64) engine = ReplacingMergeTree(version) partition by date order by date settings enable_vertical_merge_algorithm = 1, + vertical_merge_algorithm_min_rows_to_activate = 0, vertical_merge_algorithm_min_columns_to_activate = 0, min_rows_for_wide_part = 0, + min_bytes_for_wide_part = 0; insert into tab_00577 values ('2018-01-01', 2, 2), ('2018-01-01', 1, 1); insert into tab_00577 values ('2018-01-01', 0, 0); select * from tab_00577 order by version; OPTIMIZE TABLE tab_00577; select * from tab_00577; drop table tab_00577; + + +DROP TABLE IF EXISTS testCleanupR1; +CREATE TABLE testCleanupR1 (uid String, version UInt32, is_deleted UInt8) + ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{database}/tables/test_cleanup/', 'r1', version, is_deleted) + ORDER BY uid SETTINGS enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 0; +INSERT INTO testCleanupR1 (*) VALUES ('d1', 1, 0),('d2', 1, 0),('d3', 1, 0),('d4', 1, 0); +INSERT INTO testCleanupR1 (*) VALUES ('d3', 2, 1); +INSERT INTO testCleanupR1 (*) VALUES ('d1', 2, 1); +SYSTEM SYNC REPLICA testCleanupR1; -- Avoid "Cannot select parts for optimization: Entry for part all_2_2_0 hasn't been read from the replication log yet" + +OPTIMIZE TABLE testCleanupR1 FINAL CLEANUP; + +-- Only d3 to d5 remain +SELECT '== (Replicas) Test optimize =='; +SELECT * FROM testCleanupR1 order by uid; +DROP TABLE IF EXISTS testCleanupR1 \ No newline at end of file From 9f009cccd5d01be29ff8e8ab6063297ec2a73b46 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 17 Sep 2023 15:22:51 +0000 Subject: [PATCH 13/69] Incorporate review feedback --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 18 ++++++++++-------- .../MergeTree/MergeTreeIndexUSearch.cpp | 18 ++++++++++-------- 2 files changed, 20 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index f00f11359e1..d15d89ad6f9 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -160,21 +160,23 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t const auto & column_array_offsets = column_array->getOffsets(); const size_t num_rows = column_array_offsets.size(); - /// The index dimension is inferred from the inserted arrays (array cardinality). If no value was specified in the INSERT statement - /// for the annoy-indexed column (i.e. default value), we have a problem. Reject such values. - if (column_array_offsets.empty() || column_array_offsets[0] == 0) - /// (The if condition is a bit weird but I have seen either with default values) - throw Exception(ErrorCodes::INCORRECT_DATA, "Tried to insert {} rows into Annoy index but there were no values to insert. Likely, the INSERT used default values - these are not supported for Annoy.", rows_read); + /// The Annoy algorithm naturally assumes that the indexed vectors have dimension >= 0. This condition is violated if empty arrays + /// are INSERTed into an Annoy-indexed column or if no value was specified at all in which case the arrays take on their default + /// value which is also an empty array. + if (column_array->isDefaultAt(0)) + throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); /// Check all sizes are the same size_t dimension = column_array_offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension) - throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name); - /// Also check that previously inserted blocks have the same size as this block + /// Also check that previously inserted blocks have the same size as this block. + /// Note that this guarantees consistency of dimension only within parts. We are unable to detect inconsistent dimensions across + /// parts - for this, a little help from the user is needed, e.g. CONSTRAINT cnstr CHECK length(array) = 42. if (index && index->getDimensions() != dimension) - throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name); if (!index) index = std::make_shared>(dimension); diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index 9531b9188bf..a00cab6ca59 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -179,21 +179,23 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t const auto & column_array_offsets = column_array->getOffsets(); const size_t num_rows = column_array_offsets.size(); - /// The index dimension is inferred from the inserted arrays (array cardinality). If no value was specified in the INSERT statement - /// for the usearch-indexed column (i.e. default value), we have a problem. Reject such values. - if (column_array_offsets.empty() || column_array_offsets[0] == 0) - /// (The if condition is a bit weird but I have seen either with default values) - throw Exception(ErrorCodes::INCORRECT_DATA, "Tried to insert {} rows into usearch index but there were no values to insert. Likely, the INSERT used default values - these are not supported for Annoy.", rows_read); + /// The Usearch algorithm naturally assumes that the indexed vectors have dimension >= 0. This condition is violated if empty arrays + /// are INSERTed into an Usearch-indexed column or if no value was specified at all in which case the arrays take on their default + /// value which is also an empty array. + if (column_array->isDefaultAt(0)) + throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); /// Check all sizes are the same size_t dimension = column_array_offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension) - throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name); - /// Also check that previously inserted blocks have the same size as this block + /// Also check that previously inserted blocks have the same size as this block. + /// Note that this guarantees consistency of dimension only within parts. We are unable to detect inconsistent dimensions across + /// parts - for this, a little help from the user is needed, e.g. CONSTRAINT cnstr CHECK length(array) = 42. if (index && index->getDimensions() != dimension) - throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name); if (!index) index = std::make_shared>(dimension, scalar_kind); From de4f22e20aa1c4d044a525474f5ed2d03e6c59a5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 17 Sep 2023 15:26:36 +0000 Subject: [PATCH 14/69] Typo --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeIndexUSearch.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index d15d89ad6f9..3eec8614dcd 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -160,9 +160,9 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t const auto & column_array_offsets = column_array->getOffsets(); const size_t num_rows = column_array_offsets.size(); - /// The Annoy algorithm naturally assumes that the indexed vectors have dimension >= 0. This condition is violated if empty arrays + /// The Annoy algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays /// are INSERTed into an Annoy-indexed column or if no value was specified at all in which case the arrays take on their default - /// value which is also an empty array. + /// value which is also empty. if (column_array->isDefaultAt(0)) throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index a00cab6ca59..009c004faea 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -179,9 +179,9 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t const auto & column_array_offsets = column_array->getOffsets(); const size_t num_rows = column_array_offsets.size(); - /// The Usearch algorithm naturally assumes that the indexed vectors have dimension >= 0. This condition is violated if empty arrays + /// The Usearch algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays /// are INSERTed into an Usearch-indexed column or if no value was specified at all in which case the arrays take on their default - /// value which is also an empty array. + /// values which is also empty. if (column_array->isDefaultAt(0)) throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); From 262a81a7d7272523af7b601e24665deb3982eff8 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 18 Sep 2023 13:20:19 +0000 Subject: [PATCH 15/69] Fix style check for test --- .../02875_parallel_replicas_cluster_all_replicas.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql index c1cbc404aa6..1201a156246 100644 --- a/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql +++ b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql @@ -9,7 +9,7 @@ SYSTEM FLUSH LOGS; SET allow_experimental_parallel_reading_from_replicas=0; SELECT count() > 0 FROM system.text_log -WHERE query_id in (select query_id from system.query_log where log_comment = '02875_190aed82-2423-413b-ad4c-24dcca50f65b') +WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment = '02875_190aed82-2423-413b-ad4c-24dcca50f65b') AND message LIKE '%Parallel reading from replicas is disabled for cluster%'; DROP TABLE tt; From c285f85cade3c114785e5d794211a344c64a810f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 18 Sep 2023 14:43:10 +0000 Subject: [PATCH 16/69] Fix: correct check if all shards have only 1 node --- src/Interpreters/ClusterProxy/executeQuery.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 372376afc02..637d0de8a1f 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -127,12 +127,12 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, /// disable parallel replicas if cluster contains only shards with 1 replica if (context->canUseParallelReplicas()) { - bool disable_parallel_replicas = false; + bool disable_parallel_replicas = true; for (const auto & shard : cluster.getShardsInfo()) { - if (shard.getAllNodeCount() <= 1) + if (shard.getAllNodeCount() > 1) { - disable_parallel_replicas = true; + disable_parallel_replicas = false; break; } } @@ -192,10 +192,10 @@ void executeQuery( SelectStreamFactory::Shards remote_shards; auto cluster = query_info.getCluster(); - auto new_context - = updateSettingsForCluster(*cluster, context, settings, main_table, query_info.additional_filter_ast, log); - if (new_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value - != context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value) + auto new_context = updateSettingsForCluster(*cluster, context, settings, main_table, query_info.additional_filter_ast, log); + if (context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value + && context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value + != new_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value) { LOG_TRACE( log, From 13bc2941746dedc21207ed7ad4de87f63ea27cac Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 18 Sep 2023 15:09:42 +0000 Subject: [PATCH 17/69] Test for remote() --- .../02875_parallel_replicas_remote.reference | 2 ++ .../02875_parallel_replicas_remote.sql | 15 +++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/02875_parallel_replicas_remote.reference create mode 100644 tests/queries/0_stateless/02875_parallel_replicas_remote.sql diff --git a/tests/queries/0_stateless/02875_parallel_replicas_remote.reference b/tests/queries/0_stateless/02875_parallel_replicas_remote.reference new file mode 100644 index 00000000000..79ebceba739 --- /dev/null +++ b/tests/queries/0_stateless/02875_parallel_replicas_remote.reference @@ -0,0 +1,2 @@ +60 +1 diff --git a/tests/queries/0_stateless/02875_parallel_replicas_remote.sql b/tests/queries/0_stateless/02875_parallel_replicas_remote.sql new file mode 100644 index 00000000000..f47fc559df9 --- /dev/null +++ b/tests/queries/0_stateless/02875_parallel_replicas_remote.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS tt; +CREATE TABLE tt (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); +INSERT INTO tt SELECT * FROM numbers(10); + +SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, use_hedged_requests=0, parallel_replicas_for_non_replicated_merge_tree=1; +SELECT count() FROM remote('127.0.0.{1..6}', currentDatabase(), tt) settings log_comment='02875_89f3c39b-1919-48cb-b66e-ef9904e73146'; + +SYSTEM FLUSH LOGS; + +SET allow_experimental_parallel_reading_from_replicas=0; +SELECT count() > 0 FROM system.text_log +WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment = '02875_89f3c39b-1919-48cb-b66e-ef9904e73146') + AND message LIKE '%Parallel reading from replicas is disabled for cluster%'; + +DROP TABLE tt; From e1019ba3c47f175c6c582ae169517ebe4e297e28 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 18 Sep 2023 15:27:55 +0000 Subject: [PATCH 18/69] Disabling parallel replicas per shard will be done separately --- src/Processors/QueryPlan/ReadFromRemote.cpp | 51 +++++++-------------- 1 file changed, 16 insertions(+), 35 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 884bc85aaaf..f8d718b481d 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -236,49 +236,29 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact scalars["_shard_num"] = Block{{DataTypeUInt32().createColumnConst(1, shard.shard_info.shard_num), std::make_shared(), "_shard_num"}}; - ContextPtr execution_context = context; if (context->canUseParallelReplicas()) { - if (shard.shard_info.getAllNodeCount() > 1) + if (context->getSettingsRef().cluster_for_parallel_replicas.changed) { - if (context->getSettingsRef().cluster_for_parallel_replicas.changed) - { - const String cluster_for_parallel_replicas = context->getSettingsRef().cluster_for_parallel_replicas; - if (cluster_for_parallel_replicas != cluster_name) - LOG_INFO( - log, - "cluster_for_parallel_replicas has been set for the query but has no effect: {}. Distributed table cluster is " - "used: {}", - cluster_for_parallel_replicas, - cluster_name); - } - - LOG_TRACE(&Poco::Logger::get("ReadFromRemote"), "Setting `cluster_for_parallel_replicas` to {}", cluster_name); - context->setSetting("cluster_for_parallel_replicas", cluster_name); + const String cluster_for_parallel_replicas = context->getSettingsRef().cluster_for_parallel_replicas; + if (cluster_for_parallel_replicas != cluster_name) + LOG_INFO( + log, + "cluster_for_parallel_replicas has been set for the query but has no effect: {}. Distributed table cluster is " + "used: {}", + cluster_for_parallel_replicas, + cluster_name); } - else - { - ContextMutablePtr tmp = Context::createCopy(context); - tmp->setSetting("allow_experimental_parallel_reading_from_replicas", Field{0}); - execution_context = tmp; - LOG_TRACE( - &Poco::Logger::get("ReadFromRemote"), - "Parallel reading from replicas is disabled for shard. Not enough nodes: shard={} cluster={} nodes={}", - shard.shard_info.shard_num, - cluster_name, - shard.shard_info.getAllNodeCount()); - } + LOG_TRACE(log, "Setting `cluster_for_parallel_replicas` to {}", cluster_name); + context->setSetting("cluster_for_parallel_replicas", cluster_name); } - std::shared_ptr remote_query_executor; - - remote_query_executor = std::make_shared( - shard.shard_info.pool, query_string, output_stream->header, execution_context, throttler, scalars, external_tables, stage); - + auto remote_query_executor = std::make_shared( + shard.shard_info.pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage); remote_query_executor->setLogger(log); - if (execution_context->canUseParallelReplicas()) + if (context->canUseParallelReplicas()) { // when doing parallel reading from replicas (ParallelReplicasMode::READ_TASKS) on a shard: // establish a connection to a replica on the shard, the replica will instantiate coordinator to manage parallel reading from replicas on the shard. @@ -294,7 +274,8 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact if (!table_func_ptr) remote_query_executor->setMainTable(shard.main_table ? shard.main_table : main_table); - pipes.emplace_back(createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending)); + pipes.emplace_back( + createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending)); addConvertingActions(pipes.back(), output_stream->header); } From 326eaa9bc583bd0397c97760580679b5b50d46c6 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 18 Sep 2023 18:26:53 +0200 Subject: [PATCH 19/69] Better exception message in checkDataPart --- src/Storages/MergeTree/checkDataPart.cpp | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index a327ca17609..1717d91271a 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -123,9 +123,20 @@ static IMergeTreeDataPart::Checksums checkDataPart( if (data_part_storage.exists(IMergeTreeDataPart::SERIALIZATION_FILE_NAME)) { - auto serialization_file = data_part_storage.readFile(IMergeTreeDataPart::SERIALIZATION_FILE_NAME, read_settings, std::nullopt, std::nullopt); - SerializationInfo::Settings settings{ratio_of_defaults, false}; - serialization_infos = SerializationInfoByName::readJSON(columns_txt, settings, *serialization_file); + try + { + auto serialization_file = data_part_storage.readFile(IMergeTreeDataPart::SERIALIZATION_FILE_NAME, read_settings, std::nullopt, std::nullopt); + SerializationInfo::Settings settings{ratio_of_defaults, false}; + serialization_infos = SerializationInfoByName::readJSON(columns_txt, settings, *serialization_file); + } + catch (const Poco::Exception & ex) + { + throw Exception(ErrorCodes::CORRUPTED_DATA, "Failed to load {}, with error {}", IMergeTreeDataPart::SERIALIZATION_FILE_NAME, ex.message()); + } + catch (...) + { + throw; + } } auto get_serialization = [&serialization_infos](const auto & column) From 142a4631f14f02ad198e9c67700f44ec1e001dae Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 19 Sep 2023 11:22:10 +0200 Subject: [PATCH 20/69] Updated to clear current_row_sources when cleanedup rows are added --- .../Merges/Algorithms/ReplacingSortedAlgorithm.cpp | 7 ++++++- .../00577_replacing_merge_tree_vertical_merge.sql | 5 +++-- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp index 18f144bee3d..0c0598171b3 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp @@ -28,7 +28,6 @@ ReplacingSortedAlgorithm::ReplacingSortedAlgorithm( , cleanup(cleanup_) , cleanedup_rows_count(cleanedup_rows_count_) { - if (!is_deleted_column.empty()) is_deleted_column_number = header_.getPositionByName(is_deleted_column); if (!version_column.empty()) @@ -84,7 +83,10 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() if (!cleanup || !value) insertRow(); else if (cleanup && cleanedup_rows_count != nullptr) + { *cleanedup_rows_count += current_row_sources.size(); + current_row_sources.resize(0); + } } else insertRow(); @@ -142,7 +144,10 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() if (!cleanup || !value) insertRow(); else if (cleanup && cleanedup_rows_count != nullptr) + { *cleanedup_rows_count += current_row_sources.size(); + current_row_sources.resize(0); + } } else insertRow(); diff --git a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql index 931297fdd3b..e3c1bb10426 100644 --- a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql +++ b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql @@ -7,7 +7,7 @@ create table tab_00577 (date Date, version UInt64, val UInt64) engine = Replacin insert into tab_00577 values ('2018-01-01', 2, 2), ('2018-01-01', 1, 1); insert into tab_00577 values ('2018-01-01', 0, 0); select * from tab_00577 order by version; -OPTIMIZE TABLE tab_00577; +OPTIMIZE TABLE tab_00577 FINAL CLEANUP; select * from tab_00577; drop table tab_00577; @@ -15,7 +15,8 @@ drop table tab_00577; DROP TABLE IF EXISTS testCleanupR1; CREATE TABLE testCleanupR1 (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{database}/tables/test_cleanup/', 'r1', version, is_deleted) - ORDER BY uid SETTINGS enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 0; + ORDER BY uid SETTINGS enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 0, vertical_merge_algorithm_min_columns_to_activate = 0, min_rows_for_wide_part = 0, + min_bytes_for_wide_part = 0; INSERT INTO testCleanupR1 (*) VALUES ('d1', 1, 0),('d2', 1, 0),('d3', 1, 0),('d4', 1, 0); INSERT INTO testCleanupR1 (*) VALUES ('d3', 2, 1); INSERT INTO testCleanupR1 (*) VALUES ('d1', 2, 1); From 834ec7b3823a8a36ea283e30adfacc82026cf3ed Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 14 Sep 2023 09:26:30 +0800 Subject: [PATCH 21/69] reuse GeneratorJSONPath --- src/Functions/FunctionSQLJSON.h | 14 +++++++------- .../JSONPath/Generator/GeneratorJSONPath.h | 10 ++++++++++ 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index 55ffd817bb1..1786b613f98 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -199,6 +199,7 @@ public: /// Parse JSON for every row Impl impl; + GeneratorJSONPath generator_json_path(res); for (const auto i : collections::range(0, input_rows_count)) { std::string_view json{ @@ -208,7 +209,9 @@ public: bool added_to_column = false; if (document_ok) { - added_to_column = impl.insertResultToColumn(*to, document, res, context); + // Instead of creating a new generator for each row, we can reuse the same one. + generator_json_path.reinitialize(); + added_to_column = impl.insertResultToColumn(*to, document, generator_json_path, context); } if (!added_to_column) { @@ -287,9 +290,8 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const ContextPtr &) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const ContextPtr &) { - GeneratorJSONPath generator_json_path(query_ptr); Element current_element = root; VisitorStatus status; while ((status = generator_json_path.getNextItem(current_element)) != VisitorStatus::Exhausted) @@ -337,9 +339,8 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const ContextPtr & context) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const ContextPtr & context) { - GeneratorJSONPath generator_json_path(query_ptr); Element current_element = root; VisitorStatus status; @@ -405,11 +406,10 @@ public: static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const ContextPtr &) + static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath & generator_json_path, const ContextPtr &) { ColumnString & col_str = assert_cast(dest); - GeneratorJSONPath generator_json_path(query_ptr); Element current_element = root; VisitorStatus status; bool success = false; diff --git a/src/Functions/JSONPath/Generator/GeneratorJSONPath.h b/src/Functions/JSONPath/Generator/GeneratorJSONPath.h index be02656b07d..de62be98d31 100644 --- a/src/Functions/JSONPath/Generator/GeneratorJSONPath.h +++ b/src/Functions/JSONPath/Generator/GeneratorJSONPath.h @@ -105,6 +105,16 @@ public: } } + void reinitialize() + { + while (current_visitor >= 0) + { + visitors[current_visitor]->reinitialize(); + current_visitor--; + } + current_visitor = 0; + } + private: bool updateVisitorsForNextRun() { From c7ddbab9bc49757e41b737e85cbea8e1176a47bb Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 19 Sep 2023 13:06:19 +0300 Subject: [PATCH 22/69] Updated implementation --- src/DataTypes/Utils.cpp | 231 ++++++++++++++++++++++++ src/DataTypes/Utils.h | 19 ++ src/Functions/FunctionHelpers.cpp | 1 - src/Storages/MergeTree/KeyCondition.cpp | 34 +++- 4 files changed, 275 insertions(+), 10 deletions(-) create mode 100644 src/DataTypes/Utils.cpp create mode 100644 src/DataTypes/Utils.h diff --git a/src/DataTypes/Utils.cpp b/src/DataTypes/Utils.cpp new file mode 100644 index 00000000000..0168c6d256c --- /dev/null +++ b/src/DataTypes/Utils.cpp @@ -0,0 +1,231 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_type) +{ + auto from_which_type = WhichDataType(from_type->getTypeId()); + bool to_type_was_nullable = isNullableOrLowCardinalityNullable(to_type); + auto to_type_unwrapped = removeNullable(removeLowCardinality(to_type)); + + if (from_type->equals(*to_type)) + return true; + + auto to_which_type = WhichDataType(to_type->getTypeId()); + + switch (from_which_type.idx) + { + case TypeIndex::UInt8: + case TypeIndex::UInt16: + case TypeIndex::UInt32: + case TypeIndex::UInt64: + case TypeIndex::UInt128: + case TypeIndex::UInt256: + { + if (to_which_type.isUInt() && + to_type_unwrapped->getSizeOfValueInMemory() >= from_type->getSizeOfValueInMemory()) + return true; + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Int8: + case TypeIndex::Int16: + case TypeIndex::Int32: + case TypeIndex::Int64: + case TypeIndex::Int128: + case TypeIndex::Int256: + { + if (to_which_type.isInt() && + to_type_unwrapped->getSizeOfValueInMemory() >= from_type->getSizeOfValueInMemory()) + return true; + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Float32: + { + if (to_which_type.isFloat64() || to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Float64: + case TypeIndex::Date: + case TypeIndex::Date32: + case TypeIndex::DateTime: + case TypeIndex::DateTime64: + case TypeIndex::FixedString: + case TypeIndex::Enum8: + case TypeIndex::Enum16: + case TypeIndex::IPv6: + { + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Decimal32: + case TypeIndex::Decimal64: + case TypeIndex::Decimal128: + case TypeIndex::Decimal256: + { + if (to_which_type.isDecimal()) + { + auto from_type_decimal_precision = getDecimalPrecision(*from_type); + auto to_type_decimal_precision = getDecimalPrecision(*to_type_unwrapped); + if (from_type_decimal_precision > to_type_decimal_precision) + return false; + + auto from_type_decimal_scale = getDecimalScale(*from_type); + auto to_type_decimal_scale = getDecimalScale(*to_type_unwrapped); + if (from_type_decimal_scale > to_type_decimal_scale) + return false; + + return true; + } + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::UUID: + { + if (to_which_type.isUInt128() || to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::IPv4: + { + if (to_which_type.isUInt32() || to_which_type.isUInt64() || to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Nullable: + { + if (to_type_was_nullable) + { + const auto & from_type_nullable = assert_cast(*from_type); + return canBeSafelyCasted(from_type_nullable.getNestedType(), to_type_unwrapped); + } + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::LowCardinality: + { + const auto & from_type_low_cardinality = assert_cast(*from_type); + return canBeSafelyCasted(from_type_low_cardinality.getDictionaryType(), to_type_unwrapped); + } + case TypeIndex::Array: + { + if (to_which_type.isArray()) + { + const auto & from_type_array = assert_cast(*from_type); + const auto & to_type_array = assert_cast(*to_type_unwrapped); + return canBeSafelyCasted(from_type_array.getNestedType(), to_type_array.getNestedType()); + } + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Map: + { + if (to_which_type.isMap()) + { + const auto & from_type_map = assert_cast(*from_type); + const auto & to_type_map = assert_cast(*to_type_unwrapped); + if (!canBeSafelyCasted(from_type_map.getKeyType(), to_type_map.getKeyType())) + return false; + + if (!canBeSafelyCasted(from_type_map.getValueType(), to_type_map.getValueType())) + return false; + + return true; + } + + if (to_which_type.isArray()) + { + // Map nested type is Array(Tuple(key_type, value_type)) + const auto & from_type_map = assert_cast(*from_type); + const auto & to_type_array = assert_cast(*to_type_unwrapped); + const auto * to_type_nested_tuple_type = typeid_cast(to_type_array.getNestedType().get()); + if (!to_type_nested_tuple_type) + return false; + + const auto & to_type_tuple_elements = to_type_nested_tuple_type->getElements(); + if (to_type_tuple_elements.size() != 2) + return false; + + if (!canBeSafelyCasted(from_type_map.getKeyType(), to_type_tuple_elements[0])) + return false; + + if (!canBeSafelyCasted(from_type_map.getValueType(), to_type_tuple_elements[1])) + return false; + + return true; + } + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::Tuple: + { + if (to_which_type.isTuple()) + { + const auto & from_type_tuple = assert_cast(*from_type); + const auto & to_type_tuple = assert_cast(*to_type_unwrapped); + + const auto & from_tuple_type_elements = from_type_tuple.getElements(); + const auto & to_tuple_type_elements = to_type_tuple.getElements(); + + size_t lhs_type_elements_size = from_tuple_type_elements.size(); + if (lhs_type_elements_size != to_tuple_type_elements.size()) + return false; + + for (size_t i = 0; i < lhs_type_elements_size; ++i) + if (!canBeSafelyCasted(from_tuple_type_elements[i], to_tuple_type_elements[i])) + return false; + + return true; + } + + if (to_which_type.isString()) + return true; + + return false; + } + case TypeIndex::String: + case TypeIndex::Object: + case TypeIndex::Set: + case TypeIndex::Interval: + case TypeIndex::Function: + case TypeIndex::AggregateFunction: + case TypeIndex::Nothing: + return false; + } + + return true; +} + +} diff --git a/src/DataTypes/Utils.h b/src/DataTypes/Utils.h new file mode 100644 index 00000000000..bee109f4524 --- /dev/null +++ b/src/DataTypes/Utils.h @@ -0,0 +1,19 @@ +#pragma once + +#include + +namespace DB +{ + +/** Returns true if from_type can be safely casted to to_type. + * + * Examples: + * From type UInt8 to type UInt16 returns true. + * From type UInt16 to type UInt8 returns false. + * From type String to type LowCardinality(String) returns true. + * From type LowCardinality(String) to type String returns true. + * From type String to type UInt8 returns false. + */ +bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_type); + +} diff --git a/src/Functions/FunctionHelpers.cpp b/src/Functions/FunctionHelpers.cpp index 7a9817ad344..6d3c20ef2ca 100644 --- a/src/Functions/FunctionHelpers.cpp +++ b/src/Functions/FunctionHelpers.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 47521b9887b..7abd708a944 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -1257,7 +1258,7 @@ bool KeyCondition::tryPrepareSetIndex( if (!future_set) return false; - const auto & set_types = future_set->getTypes(); + const auto set_types = future_set->getTypes(); size_t set_types_size = set_types.size(); size_t indexes_mapping_size = indexes_mapping.size(); @@ -1283,24 +1284,37 @@ bool KeyCondition::tryPrepareSetIndex( for (size_t indexes_mapping_index = 0; indexes_mapping_index < indexes_mapping_size; ++indexes_mapping_index) { + const auto & key_column_type = data_types[indexes_mapping_index]; size_t set_element_index = indexes_mapping[indexes_mapping_index].tuple_index; - const auto & set_element_type = set_types[set_element_index]; - auto & set_column = set_columns[set_element_index]; + auto set_element_type = set_types[set_element_index]; + auto set_column = set_columns[set_element_index]; - bool is_set_column_nullable = set_element_type->isNullable(); - bool is_set_column_low_cardinality_nullable = set_element_type->isLowCardinalityNullable(); + if (canBeSafelyCasted(set_element_type, key_column_type)) + { + set_columns[set_element_index] = castColumn({set_column, set_element_type, {}}, key_column_type); + continue; + } + + if (!key_column_type->canBeInsideNullable()) + return false; const NullMap * set_column_null_map = nullptr; - if (is_set_column_nullable || is_set_column_low_cardinality_nullable) + if (isNullableOrLowCardinalityNullable(set_element_type)) { - if (is_set_column_low_cardinality_nullable) + if (WhichDataType(set_element_type).isLowCardinality()) + { + set_element_type = removeLowCardinality(set_element_type); set_column = set_column->convertToFullColumnIfLowCardinality(); + } - set_column_null_map = &assert_cast(*set_column).getNullMapData(); + set_element_type = removeNullable(set_element_type); + const auto & set_column_nullable = assert_cast(*set_column); + set_column_null_map = &set_column_nullable.getNullMapData(); + set_column = set_column_nullable.getNestedColumnPtr(); } - auto nullable_set_column = castColumnAccurateOrNull({set_column, set_element_type, {}}, data_types[indexes_mapping_index]); + auto nullable_set_column = castColumnAccurateOrNull({set_column, set_element_type, {}}, key_column_type); const auto & nullable_set_column_typed = assert_cast(*nullable_set_column); const auto & nullable_set_column_null_map = nullable_set_column_typed.getNullMapData(); size_t nullable_set_column_null_map_size = nullable_set_column_null_map.size(); @@ -1321,6 +1335,8 @@ bool KeyCondition::tryPrepareSetIndex( set_column = nullable_set_column_typed.getNestedColumn().filter(filter, 0); } + + set_columns[set_element_index] = std::move(set_column); } out.set_index = std::make_shared(set_columns, std::move(indexes_mapping)); From 363ee7cae58c4118452bd856406d55469dae9e94 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Sep 2023 13:41:55 +0200 Subject: [PATCH 23/69] Better check for exceptions in checkData of non replicated merge tree --- src/Storages/MergeTree/checkDataPart.cpp | 46 ++++++++++++++++++++++++ src/Storages/MergeTree/checkDataPart.h | 2 +- src/Storages/StorageMergeTree.cpp | 14 +++++--- 3 files changed, 57 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 1717d91271a..6d8e02701fa 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -15,6 +15,11 @@ #include #include #include +#include + +#if USE_AZURE_BLOB_STORAGE +# include +#endif namespace CurrentMetrics { @@ -68,6 +73,47 @@ bool isRetryableException(const Exception & e) return false; } +bool isRetryableException(const std::exception_ptr exception_ptr) +{ + try + { + rethrow_exception(exception_ptr); + } +#if USE_AWS_S3 + catch (const S3Exception & s3_exception) + { + if (s3_exception.isRetryableError()) + return true; + } +#endif +#if USE_AZURE_BLOB_STORAGE + catch (const Azure::Core::RequestFailedException &) + { + return true; + } +#endif + catch (const Exception & e) + { + if (isNotEnoughMemoryErrorCode(e.code())) + return true; + + if (e.code() == ErrorCodes::NETWORK_ERROR || e.code() == ErrorCodes::SOCKET_TIMEOUT) + return true; + } + catch (const Poco::Net::NetException &) + { + return true; + } + catch (const Poco::TimeoutException &) + { + return true; + } + + /// In fact, there can be other similar situations. + /// But it is OK, because there is a safety guard against deleting too many parts. + return false; +} + static IMergeTreeDataPart::Checksums checkDataPart( MergeTreeData::DataPartPtr data_part, diff --git a/src/Storages/MergeTree/checkDataPart.h b/src/Storages/MergeTree/checkDataPart.h index 20ddecad3ed..88e86db37fc 100644 --- a/src/Storages/MergeTree/checkDataPart.h +++ b/src/Storages/MergeTree/checkDataPart.h @@ -14,5 +14,5 @@ IMergeTreeDataPart::Checksums checkDataPart( bool isNotEnoughMemoryErrorCode(int code); bool isRetryableException(const Exception & e); - +bool isRetryableException(const std::exception_ptr exception_ptr); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 4f27cbc76ed..b2e6e6d33fa 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2222,10 +2222,13 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ results.emplace_back(part->name, true, "Checksums recounted and written to disk."); } - catch (const Exception & ex) + catch (...) { + if (isRetryableException(std::current_exception())) + throw; + tryLogCurrentException(log, __PRETTY_FUNCTION__); - results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + results.emplace_back(part->name, false, "Check of part finished with error: '" + getCurrentExceptionMessage(false) + "'"); } } else @@ -2235,9 +2238,12 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ checkDataPart(part, true); results.emplace_back(part->name, true, ""); } - catch (const Exception & ex) + catch (...) { - results.emplace_back(part->name, false, ex.message()); + if (isRetryableException(std::current_exception())) + throw; + + results.emplace_back(part->name, false, getCurrentExceptionMessage(false)); } } } From 7b1cf9d912c00eb26f9d368f2fcc9742a74249eb Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Sep 2023 13:45:34 +0200 Subject: [PATCH 24/69] Revert "Better check for exceptions in checkData of non replicated merge tree" This reverts commit 363ee7cae58c4118452bd856406d55469dae9e94. --- src/Storages/MergeTree/checkDataPart.cpp | 46 ------------------------ src/Storages/MergeTree/checkDataPart.h | 2 +- src/Storages/StorageMergeTree.cpp | 14 +++----- 3 files changed, 5 insertions(+), 57 deletions(-) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 6d8e02701fa..1717d91271a 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -15,11 +15,6 @@ #include #include #include -#include - -#if USE_AZURE_BLOB_STORAGE -# include -#endif namespace CurrentMetrics { @@ -73,47 +68,6 @@ bool isRetryableException(const Exception & e) return false; } -bool isRetryableException(const std::exception_ptr exception_ptr) -{ - try - { - rethrow_exception(exception_ptr); - } -#if USE_AWS_S3 - catch (const S3Exception & s3_exception) - { - if (s3_exception.isRetryableError()) - return true; - } -#endif -#if USE_AZURE_BLOB_STORAGE - catch (const Azure::Core::RequestFailedException &) - { - return true; - } -#endif - catch (const Exception & e) - { - if (isNotEnoughMemoryErrorCode(e.code())) - return true; - - if (e.code() == ErrorCodes::NETWORK_ERROR || e.code() == ErrorCodes::SOCKET_TIMEOUT) - return true; - } - catch (const Poco::Net::NetException &) - { - return true; - } - catch (const Poco::TimeoutException &) - { - return true; - } - - /// In fact, there can be other similar situations. - /// But it is OK, because there is a safety guard against deleting too many parts. - return false; -} - static IMergeTreeDataPart::Checksums checkDataPart( MergeTreeData::DataPartPtr data_part, diff --git a/src/Storages/MergeTree/checkDataPart.h b/src/Storages/MergeTree/checkDataPart.h index 88e86db37fc..20ddecad3ed 100644 --- a/src/Storages/MergeTree/checkDataPart.h +++ b/src/Storages/MergeTree/checkDataPart.h @@ -14,5 +14,5 @@ IMergeTreeDataPart::Checksums checkDataPart( bool isNotEnoughMemoryErrorCode(int code); bool isRetryableException(const Exception & e); -bool isRetryableException(const std::exception_ptr exception_ptr); + } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b2e6e6d33fa..4f27cbc76ed 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2222,13 +2222,10 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ results.emplace_back(part->name, true, "Checksums recounted and written to disk."); } - catch (...) + catch (const Exception & ex) { - if (isRetryableException(std::current_exception())) - throw; - tryLogCurrentException(log, __PRETTY_FUNCTION__); - results.emplace_back(part->name, false, "Check of part finished with error: '" + getCurrentExceptionMessage(false) + "'"); + results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); } } else @@ -2238,12 +2235,9 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ checkDataPart(part, true); results.emplace_back(part->name, true, ""); } - catch (...) + catch (const Exception & ex) { - if (isRetryableException(std::current_exception())) - throw; - - results.emplace_back(part->name, false, getCurrentExceptionMessage(false)); + results.emplace_back(part->name, false, ex.message()); } } } From 9dcc41cf83d71f014c7e612c9653c1c789ca6d34 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Sep 2023 13:47:32 +0200 Subject: [PATCH 25/69] Fxi --- src/Storages/StorageMergeTree.cpp | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index bae91ec8bb1..22700712829 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2231,10 +2231,13 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ results.emplace_back(part->name, true, "Checksums recounted and written to disk."); } - catch (const Exception & ex) + catch (...) { + if (isRetryableException(std::current_exception())) + throw; + tryLogCurrentException(log, __PRETTY_FUNCTION__); - results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + results.emplace_back(part->name, false, "Check of part finished with error: '" + getCurrentExceptionMessage(false) + "'"); } } else @@ -2244,9 +2247,12 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ checkDataPart(part, true); results.emplace_back(part->name, true, ""); } - catch (const Exception & ex) + catch (...) { - results.emplace_back(part->name, false, ex.message()); + if (isRetryableException(std::current_exception())) + throw; + + results.emplace_back(part->name, false, getCurrentExceptionMessage(false)); } } } From c66b60f00f851670a36789a45254a4c07ee85e47 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 19 Sep 2023 07:12:14 -0700 Subject: [PATCH 26/69] Fix SimHash function issue for s390x --- src/Functions/FunctionsStringHash.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsStringHash.cpp b/src/Functions/FunctionsStringHash.cpp index ff8ff2d2651..0bf6e39e651 100644 --- a/src/Functions/FunctionsStringHash.cpp +++ b/src/Functions/FunctionsStringHash.cpp @@ -18,6 +18,10 @@ #include "vec_crc32.h" #endif +#if defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__ +#include +#endif + namespace DB { @@ -43,7 +47,7 @@ struct Hash #elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ return crc32_ppc(crc, reinterpret_cast(&val), sizeof(val)); #elif defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__ - return s390x_crc32(crc, val); + return crc32c_le(static_cast(crc), reinterpret_cast(&val), sizeof(val)); #else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "String hash is not implemented without sse4.2 support"); #endif @@ -58,7 +62,7 @@ struct Hash #elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ return crc32_ppc(crc, reinterpret_cast(&val), sizeof(val)); #elif defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__ - return s390x_crc32_u32(crc, val); + return crc32c_le(static_cast(crc), reinterpret_cast(&val), sizeof(val)); #else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "String hash is not implemented without sse4.2 support"); #endif @@ -73,7 +77,7 @@ struct Hash #elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ return crc32_ppc(crc, reinterpret_cast(&val), sizeof(val)); #elif defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__ - return s390x_crc32_u16(crc, val); + return crc32c_le(static_cast(crc), reinterpret_cast(&val), sizeof(val)); #else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "String hash is not implemented without sse4.2 support"); #endif @@ -88,7 +92,7 @@ struct Hash #elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ return crc32_ppc(crc, reinterpret_cast(&val), sizeof(val)); #elif defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__ - return s390x_crc32_u8(crc, val); + return crc32c_le(static_cast(crc), reinterpret_cast(&val), sizeof(val)); #else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "String hash is not implemented without sse4.2 support"); #endif From 27d8eefe9376ba73ee8a898ec674d324b1a91d7c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 19 Sep 2023 17:14:32 +0300 Subject: [PATCH 27/69] Fixed tests --- src/DataTypes/Utils.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/Utils.cpp b/src/DataTypes/Utils.cpp index 0168c6d256c..1c36b2dfc9d 100644 --- a/src/DataTypes/Utils.cpp +++ b/src/DataTypes/Utils.cpp @@ -15,10 +15,10 @@ bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_typ bool to_type_was_nullable = isNullableOrLowCardinalityNullable(to_type); auto to_type_unwrapped = removeNullable(removeLowCardinality(to_type)); - if (from_type->equals(*to_type)) + if (from_type->equals(*to_type_unwrapped)) return true; - auto to_which_type = WhichDataType(to_type->getTypeId()); + auto to_which_type = WhichDataType(to_type_unwrapped->getTypeId()); switch (from_which_type.idx) { From a15029ecb593e218cabeb3ae2af5d2afa3f22c6e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 19 Sep 2023 16:29:02 +0000 Subject: [PATCH 28/69] Query Cache: Reject queries with non-deterministic functions by default https://github.com/ClickHouse/support-escalation/issues/963 --- docs/en/operations/query-cache.md | 4 ++-- src/Common/ErrorCodes.cpp | 1 + src/Interpreters/executeQuery.cpp | 14 +++++++++----- ...uery_cache_nondeterministic_functions.reference | 1 - ...2494_query_cache_nondeterministic_functions.sql | 6 +++--- 5 files changed, 15 insertions(+), 11 deletions(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index bbde77338af..e111206355e 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -142,7 +142,7 @@ As a result, the query cache stores for each query multiple (partial) result blocks. While this behavior is a good default, it can be suppressed using setting [query_cache_squash_partial_results](settings/settings.md#query-cache-squash-partial-results). -Also, results of queries with non-deterministic functions are not cached. Such functions include +Also, results of queries with non-deterministic functions are not cached by default. Such functions include - functions for accessing dictionaries: [`dictGet()`](../sql-reference/functions/ext-dict-functions.md#dictGet) etc. - [user-defined functions](../sql-reference/statements/create/function.md), - functions which return the current date or time: [`now()`](../sql-reference/functions/date-time-functions.md#now), @@ -158,7 +158,7 @@ Also, results of queries with non-deterministic functions are not cached. Such f - functions which depend on the environment: [`currentUser()`](../sql-reference/functions/other-functions.md#currentUser), [`queryID()`](../sql-reference/functions/other-functions.md#queryID), [`getMacro()`](../sql-reference/functions/other-functions.md#getMacro) etc. -Caching of non-deterministic functions can be forced regardless using setting +To force caching of results of queries with non-deterministic functionsregardless, using setting [query_cache_store_results_of_queries_with_nondeterministic_functions](settings/settings.md#query-cache-store-results-of-queries-with-nondeterministic-functions). Finally, entries in the query cache are not shared between users due to security reasons. For example, user A must not be able to bypass a diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f23685c37d1..ad34516b00e 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -585,6 +585,7 @@ M(700, USER_SESSION_LIMIT_EXCEEDED) \ M(701, CLUSTER_DOESNT_EXIST) \ M(702, CLIENT_INFO_DOES_NOT_MATCH) \ + M(703, CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 310af2f9812..9b24b5df9b2 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -94,11 +94,12 @@ namespace DB namespace ErrorCodes { + extern const int CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS; extern const int INTO_OUTFILE_NOT_ALLOWED; - extern const int QUERY_WAS_CANCELLED; extern const int INVALID_TRANSACTION; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; + extern const int QUERY_WAS_CANCELLED; } @@ -991,7 +992,7 @@ static std::tuple executeQueryImpl( if (!async_insert) { - /// If it is a non-internal SELECT, and passive/read use of the query cache is enabled, and the cache knows the query, then set + /// If it is a non-internal SELECT, and passive (read) use of the query cache is enabled, and the cache knows the query, then set /// a pipeline with a source populated by the query cache. auto get_result_from_query_cache = [&]() { @@ -1091,11 +1092,14 @@ static std::tuple executeQueryImpl( res = interpreter->execute(); - /// If it is a non-internal SELECT query, and active/write use of the query cache is enabled, then add a processor on + /// If it is a non-internal SELECT query, and active (write) use of the query cache is enabled, then add a processor on /// top of the pipeline which stores the result in the query cache. - if (can_use_query_cache && settings.enable_writes_to_query_cache - && (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions)) + if (can_use_query_cache && settings.enable_writes_to_query_cache) { + if (astContainsNonDeterministicFunctions(ast, context) && !settings.query_cache_store_results_of_queries_with_nondeterministic_functions) + throw Exception(ErrorCodes::CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS, + "Unable to cache the query result because the query contains a non-deterministic function. Use setting query_cache_store_results_of_queries_with_nondeterministic_functions = 1 to store the query result regardless."); + QueryCache::Key key( ast, res.pipeline.getHeader(), context->getUserName(), settings.query_cache_share_between_users, diff --git a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.reference b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.reference index cb6165c307a..e666f54d4c4 100644 --- a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.reference +++ b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.reference @@ -1,4 +1,3 @@ -1 0 --- 1 diff --git a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql index 045b7258a34..3a2e24d6bfe 100644 --- a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql +++ b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql @@ -3,13 +3,13 @@ SYSTEM DROP QUERY CACHE; --- rand() is non-deterministic, with default settings no entry in the query cache should be created -SELECT COUNT(rand(1)) SETTINGS use_query_cache = true; +-- rand() is non-deterministic, the query is rejected by default +SELECT COUNT(rand(1)) SETTINGS use_query_cache = true; -- { serverError 703 } SELECT COUNT(*) FROM system.query_cache; SELECT '---'; --- But an entry can be forced using a setting +-- Force caching using a setting SELECT COUNT(RAND(1)) SETTINGS use_query_cache = true, query_cache_store_results_of_queries_with_nondeterministic_functions = true; SELECT COUNT(*) FROM system.query_cache; From accbe24e08a92254c062f46d57e72a700334f10e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Sep 2023 18:42:46 +0200 Subject: [PATCH 29/69] Fix test --- tests/integration/test_check_table/test.py | 33 ++++++++-------------- 1 file changed, 12 insertions(+), 21 deletions(-) diff --git a/tests/integration/test_check_table/test.py b/tests/integration/test_check_table/test.py index 613ac3fb35f..99a5846d4ee 100644 --- a/tests/integration/test_check_table/test.py +++ b/tests/integration/test_check_table/test.py @@ -109,21 +109,15 @@ def test_check_normal_table_corruption(started_cluster): corrupt_data_part_on_disk(node1, "non_replicated_mt", "201902_1_1_0") - assert ( - node1.query( - "CHECK TABLE non_replicated_mt", - settings={"check_query_single_value_result": 0}, - ).strip() - == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 25." - ) + assert node1.query( + "CHECK TABLE non_replicated_mt", + settings={"check_query_single_value_result": 0}, + ).strip().split("\t")[0:2] == ["201902_1_1_0", "0"] - assert ( - node1.query( - "CHECK TABLE non_replicated_mt", - settings={"check_query_single_value_result": 0}, - ).strip() - == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 25." - ) + assert node1.query( + "CHECK TABLE non_replicated_mt", + settings={"check_query_single_value_result": 0}, + ).strip().split("\t")[0:2] == ["201902_1_1_0", "0"] node1.query( "INSERT INTO non_replicated_mt VALUES (toDate('2019-01-01'), 1, 10), (toDate('2019-01-01'), 2, 12)" @@ -141,13 +135,10 @@ def test_check_normal_table_corruption(started_cluster): remove_checksums_on_disk(node1, "non_replicated_mt", "201901_2_2_0") - assert ( - node1.query( - "CHECK TABLE non_replicated_mt PARTITION 201901", - settings={"check_query_single_value_result": 0}, - ) - == "201901_2_2_0\t0\tCheck of part finished with error: \\'Cannot read all data. Bytes read: 2. Bytes expected: 25.\\'\n" - ) + assert node1.query( + "CHECK TABLE non_replicated_mt PARTITION 201901", + settings={"check_query_single_value_result": 0}, + ).strip().split("\t")[0:2] == ["201901_2_2_0", "0"] def test_check_replicated_table_simple(started_cluster): From 5111f1e0901f13a758408a7aee39baba586eeb29 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 19 Sep 2023 18:59:00 +0200 Subject: [PATCH 30/69] Update docs/en/operations/query-cache.md Co-authored-by: Nikita Taranov --- docs/en/operations/query-cache.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index e111206355e..6e21b0b3658 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -158,7 +158,7 @@ Also, results of queries with non-deterministic functions are not cached by defa - functions which depend on the environment: [`currentUser()`](../sql-reference/functions/other-functions.md#currentUser), [`queryID()`](../sql-reference/functions/other-functions.md#queryID), [`getMacro()`](../sql-reference/functions/other-functions.md#getMacro) etc. -To force caching of results of queries with non-deterministic functionsregardless, using setting +To force caching of results of queries with non-deterministic functions regardless, use setting [query_cache_store_results_of_queries_with_nondeterministic_functions](settings/settings.md#query-cache-store-results-of-queries-with-nondeterministic-functions). Finally, entries in the query cache are not shared between users due to security reasons. For example, user A must not be able to bypass a From fa51399ede28f551a62061d6961032d1c99e763a Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 19 Sep 2023 21:39:59 +0000 Subject: [PATCH 31/69] Make clickhouse-local logging (server_logs_file) prepend timestamps etc --- programs/local/LocalServer.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index d56e7d8b47c..9fb629a0871 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -32,6 +32,8 @@ #include #include #include +#include +#include #include #include #include @@ -599,7 +601,9 @@ void LocalServer::processConfig() { auto poco_logs_level = Poco::Logger::parseLevel(level); Poco::Logger::root().setLevel(poco_logs_level); - Poco::Logger::root().setChannel(Poco::AutoPtr(new Poco::SimpleFileChannel(server_logs_file))); + Poco::AutoPtr pf = new OwnPatternFormatter; + Poco::AutoPtr log = new OwnFormattingChannel(pf, new Poco::SimpleFileChannel(server_logs_file)); + Poco::Logger::root().setChannel(log); logging_initialized = true; } else if (logging || is_interactive) From c856ec4087bb990c6fe1c22d6b619a6ae8646e31 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 19 Sep 2023 21:43:44 +0000 Subject: [PATCH 32/69] Prevent ParquetMetadata reading 40 MB from each file unnecessarily --- .../Formats/Impl/ParquetMetadataInputFormat.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp index 043e6d2260c..1f81f5ac201 100644 --- a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp @@ -130,7 +130,7 @@ static std::shared_ptr getFileMetadata( const FormatSettings & format_settings, std::atomic & is_stopped) { - auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES); + auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); return parquet::ReadMetaData(arrow_file); } @@ -495,12 +495,15 @@ NamesAndTypesList ParquetMetadataSchemaReader::readSchema() void registerInputFormatParquetMetadata(FormatFactory & factory) { - factory.registerInputFormat( + factory.registerRandomAccessInputFormat( "ParquetMetadata", - [](ReadBuffer &buf, - const Block &sample, - const RowInputFormatParams &, - const FormatSettings & settings) + [](ReadBuffer & buf, + const Block & sample, + const FormatSettings & settings, + const ReadSettings &, + bool /* is_remote_fs */, + size_t /* max_download_threads */, + size_t /* max_parsing_threads */) { return std::make_shared(buf, sample, settings); }); From e8cd42945388ecc20192b398da17867955873160 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 19 Sep 2023 22:11:58 +0000 Subject: [PATCH 33/69] add error code INVALID_IDENTIFIER --- src/Common/ErrorCodes.cpp | 1 + src/Interpreters/DatabaseAndTableWithAlias.cpp | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f23685c37d1..e2ad8741ca3 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -585,6 +585,7 @@ M(700, USER_SESSION_LIMIT_EXCEEDED) \ M(701, CLUSTER_DOESNT_EXIST) \ M(702, CLIENT_INFO_DOES_NOT_MATCH) \ + M(703, INVALID_IDENTIFIER) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Interpreters/DatabaseAndTableWithAlias.cpp b/src/Interpreters/DatabaseAndTableWithAlias.cpp index 9b6ce4f22d3..cb993611ecb 100644 --- a/src/Interpreters/DatabaseAndTableWithAlias.cpp +++ b/src/Interpreters/DatabaseAndTableWithAlias.cpp @@ -16,6 +16,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int INVALID_IDENTIFIER; } DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableIdentifier & identifier, const String & current_database) @@ -37,7 +38,7 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & ident else if (identifier.name_parts.size() == 1) table = identifier.name_parts[0]; else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: invalid identifier"); + throw Exception(ErrorCodes::INVALID_IDENTIFIER, "Invalid identifier"); if (database.empty()) database = current_database; @@ -50,7 +51,7 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTPtr & node, const else if (const auto * identifier = node->as()) *this = DatabaseAndTableWithAlias(*identifier, current_database); else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: identifier or table identifier expected"); + throw Exception(ErrorCodes::INVALID_IDENTIFIER, "Identifier or table identifier expected"); } DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database) From e9e5c7a09bdfc36459d7a2ac141914a9357c67f7 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 20 Sep 2023 08:44:30 +0800 Subject: [PATCH 34/69] update --- src/Functions/FunctionSQLJSON.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index 1786b613f98..0533f3d419a 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -209,7 +209,7 @@ public: bool added_to_column = false; if (document_ok) { - // Instead of creating a new generator for each row, we can reuse the same one. + /// Instead of creating a new generator for each row, we can reuse the same one. generator_json_path.reinitialize(); added_to_column = impl.insertResultToColumn(*to, document, generator_json_path, context); } From 34aecc0bf30ff4f6109d593ddc305f45324ba59a Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 19 Sep 2023 23:05:48 -0400 Subject: [PATCH 35/69] Adjusting `num_streams` by expected work in StorageS3 --- src/Storages/StorageS3.cpp | 74 ++++++++++++++++++++++++++++++++++++-- src/Storages/StorageS3.h | 14 ++++++-- 2 files changed, 83 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 341d8b3f768..e99be7a1204 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -180,6 +180,13 @@ public: return nextAssumeLocked(); } + size_t objectsCount() + { + assert(outcome_future.valid()); + first_outcome = outcome_future.get(); + return first_outcome->GetResult().GetContents().size(); + } + ~Impl() { list_objects_pool.wait(); @@ -225,8 +232,17 @@ private: { buffer.clear(); - assert(outcome_future.valid()); - auto outcome = outcome_future.get(); + ListObjectsOutcome outcome; + if (unlikely(first_outcome)) + { + outcome = std::move(*first_outcome); + first_outcome = std::nullopt; + } + else + { + assert(outcome_future.valid()); + outcome = outcome_future.get(); + } if (!outcome.IsSuccess()) { @@ -343,6 +359,7 @@ private: ThreadPool list_objects_pool; ThreadPoolCallbackRunner list_objects_scheduler; std::future outcome_future; + std::optional first_outcome; /// the result will be set by `estimatedKeysCount` std::function file_progress_callback; }; @@ -364,6 +381,11 @@ StorageS3Source::KeyWithInfo StorageS3Source::DisclosedGlobIterator::next() return pimpl->next(); } +size_t StorageS3Source::DisclosedGlobIterator::estimatedKeysCount() +{ + return pimpl->objectsCount(); +} + class StorageS3Source::KeysIterator::Impl : WithContext { public: @@ -425,6 +447,11 @@ public: return {key, info}; } + size_t objectsCount() + { + return keys.size(); + } + private: Strings keys; std::atomic_size_t index = 0; @@ -459,6 +486,43 @@ StorageS3Source::KeyWithInfo StorageS3Source::KeysIterator::next() return pimpl->next(); } +size_t StorageS3Source::KeysIterator::estimatedKeysCount() +{ + return pimpl->objectsCount(); +} + +StorageS3Source::ReadTaskIterator::ReadTaskIterator( + const DB::ReadTaskCallback & callback_, + const size_t max_threads_count) + : callback(callback_) +{ + ThreadPool pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, max_threads_count); + auto pool_scheduler = threadPoolCallbackRunner(pool, "ReadTaskIteratorPrefetch"); + + std::vector> keys; + for (size_t i = 0; i < max_threads_count; ++i) + keys.push_back(pool_scheduler([this] { return callback(); }, Priority{})); + + pool.wait(); + buffer.reserve(max_threads_count); + for (auto & key_future : keys) + buffer.emplace_back(key_future.get(), std::nullopt); +} + +StorageS3Source::KeyWithInfo StorageS3Source::ReadTaskIterator::next() +{ + size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + if (current_index >= buffer.size()) + return {callback(), {}}; + + return buffer[current_index]; +} + +size_t StorageS3Source::ReadTaskIterator::estimatedKeysCount() +{ + return buffer.size(); +} + StorageS3Source::StorageS3Source( const ReadFromFormatInfo & info, const String & format_, @@ -965,7 +1029,7 @@ std::shared_ptr StorageS3::createFileIterator( { if (distributed_processing) { - return std::make_shared(local_context->getReadTaskCallback()); + return std::make_shared(local_context->getReadTaskCallback(), local_context->getSettingsRef().max_threads); } else if (configuration.withGlobs()) { @@ -1017,6 +1081,9 @@ Pipe StorageS3::read( std::shared_ptr iterator_wrapper = createFileIterator( query_configuration, distributed_processing, local_context, query_info.query, virtual_columns, nullptr, local_context->getFileProgressCallback()); + size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); + num_streams = std::min(num_streams, estimated_keys_count); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && local_context->getSettingsRef().optimize_count_from_files; @@ -1024,6 +1091,7 @@ Pipe StorageS3::read( const size_t max_threads = local_context->getSettingsRef().max_threads; const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / num_streams); + pipes.reserve(num_streams); for (size_t i = 0; i < num_streams; ++i) { pipes.emplace_back(std::make_shared( diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index ee03b9f18c2..f0315244088 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -60,6 +60,10 @@ public: virtual ~IIterator() = default; virtual KeyWithInfo next() = 0; + /// Estimates how many streams we need to process all files. + /// If keys count >= max_threads_count, the returned number may not represent the actual number of the keys. + virtual size_t estimatedKeysCount() = 0; + KeyWithInfo operator ()() { return next(); } }; @@ -77,6 +81,7 @@ public: std::function progress_callback_ = {}); KeyWithInfo next() override; + size_t estimatedKeysCount() override; private: class Impl; @@ -100,6 +105,7 @@ public: std::function progress_callback_ = {}); KeyWithInfo next() override; + size_t estimatedKeysCount() override; private: class Impl; @@ -110,11 +116,15 @@ public: class ReadTaskIterator : public IIterator { public: - explicit ReadTaskIterator(const ReadTaskCallback & callback_) : callback(callback_) {} + explicit ReadTaskIterator(const ReadTaskCallback & callback_, const size_t max_threads_count); - KeyWithInfo next() override { return {callback(), {}}; } + KeyWithInfo next() override; + size_t estimatedKeysCount() override; private: + KeysWithInfo buffer; + std::atomic_size_t index = 0; + ReadTaskCallback callback; }; From 56dca144cc4a35c953113ab0aa7f0bf3448cbf73 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 20 Sep 2023 07:09:31 +0200 Subject: [PATCH 36/69] Fix test_backup_restore_on_cluster/test.py::test_stop_other_host_during_backup flakiness Signed-off-by: Azat Khuzhin --- tests/integration/test_backup_restore_on_cluster/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index dfce2f15413..20f538cca58 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -1096,6 +1096,7 @@ def test_stop_other_host_during_backup(kill): if status == "BACKUP_CREATED": node1.query("DROP TABLE tbl ON CLUSTER 'cluster' SYNC") node1.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") + node1.query("SYSTEM SYNC REPLICA tbl") assert node1.query("SELECT * FROM tbl ORDER BY x") == TSV([3, 5]) elif status == "BACKUP_FAILED": assert not os.path.exists( From 074ee40e3b80010a6f77765423ff153be79b10a2 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 20 Sep 2023 06:55:39 +0000 Subject: [PATCH 37/69] avoid excessive calls to getifaddrs Signed-off-by: Duc Canh Le --- src/Common/isLocalAddress.cpp | 29 +++++++++++++++++++++++++++-- 1 file changed, 27 insertions(+), 2 deletions(-) diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index 7569c6fc14e..8089a713eb8 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -2,6 +2,9 @@ #include #include +#include +#include +#include #include #include #include @@ -74,6 +77,29 @@ struct NetworkInterfaces { freeifaddrs(ifaddr); } + + static NetworkInterfaces & instance() + { + static constexpr int NET_INTERFACE_VALID_PERIOD_SECONDS = 30; + static std::unique_ptr nf = std::make_unique(); + static time_t last_updated_time = time(nullptr); + static std::shared_mutex nf_mtx; + + time_t now = time(nullptr); + + if (now - last_updated_time > NET_INTERFACE_VALID_PERIOD_SECONDS) + { + std::unique_lock lock(nf_mtx); + nf = std::make_unique(); + last_updated_time = now; + return *nf; + } + else + { + std::shared_lock lock(nf_mtx); + return *nf; + } + } }; } @@ -111,8 +137,7 @@ bool isLocalAddress(const Poco::Net::IPAddress & address) } } - NetworkInterfaces interfaces; - return interfaces.hasAddress(address); + return NetworkInterfaces::instance().hasAddress(address); } From b1bddae5ce7c653ade6405321bf9f9c4cd991f23 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 20 Sep 2023 10:26:29 +0200 Subject: [PATCH 38/69] Rewrite the test to check the query plan too. --- ...optimize_read_in_order_from_view.reference | 4 +- .../02877_optimize_read_in_order_from_view.sh | 40 +++++++++++++++++++ ...02877_optimize_read_in_order_from_view.sql | 21 ---------- 3 files changed, 43 insertions(+), 22 deletions(-) create mode 100755 tests/queries/0_stateless/02877_optimize_read_in_order_from_view.sh delete mode 100644 tests/queries/0_stateless/02877_optimize_read_in_order_from_view.sql diff --git a/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.reference b/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.reference index 05893173f2b..0f7d306c7b5 100644 --- a/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.reference +++ b/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.reference @@ -18,4 +18,6 @@ 283 282 281 -read_rows=ok +read_rows:ok +ReadFromMergeTree (default.table1) +ReadType: InReverseOrder diff --git a/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.sh b/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.sh new file mode 100755 index 00000000000..a3a906a9891 --- /dev/null +++ b/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.sh @@ -0,0 +1,40 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh +set -eo pipefail + +$CLICKHOUSE_CLIENT --multiquery < 40 )); then + echo "read_rows:$read_rows" +else + echo "read_rows:ok" +fi + +query_plan=$($CLICKHOUSE_CLIENT -q "EXPLAIN actions=1 $query" --optimize_read_in_order=1) + +echo "$query_plan" | grep -A 1 "ReadFromMergeTree" | sed 's/^[ \t]*//' + +$CLICKHOUSE_CLIENT -q "DROP TABLE view1" +$CLICKHOUSE_CLIENT -q "DROP TABLE table1" diff --git a/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.sql b/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.sql deleted file mode 100644 index 3397681d0d9..00000000000 --- a/tests/queries/0_stateless/02877_optimize_read_in_order_from_view.sql +++ /dev/null @@ -1,21 +0,0 @@ -SET optimize_read_in_order=1; - -DROP TABLE IF EXISTS view1; -DROP TABLE IF EXISTS table1; - -CREATE TABLE table1 (number UInt64) ENGINE=MergeTree ORDER BY number SETTINGS index_granularity=1; -INSERT INTO table1 SELECT number FROM numbers(1, 300); - -CREATE VIEW view1 AS SELECT number FROM table1; - --- The following SELECT is expected to read 20 rows. In fact it may decide to read more than 20 rows, but not too many anyway. --- So we'll check that the number of read rows is less than 40. - -SELECT /* test 02877, query 1 */ * FROM (SELECT * FROM view1) ORDER BY number DESC LIMIT 20 SETTINGS log_queries=1; - -SYSTEM FLUSH LOGS; - -SELECT concat('read_rows=', if(read_rows<40, 'ok', toString(read_rows))) FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE '%test 02877, query 1%' AND type='QueryFinish'; - -DROP TABLE view1; -DROP TABLE table1; From 5ace2a15db1671eddd32c7305e4ebe50c40cab93 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 20 Sep 2023 16:46:32 +0800 Subject: [PATCH 39/69] Update src/Common/isLocalAddress.cpp Co-authored-by: Igor Nikonov <954088+devcrafter@users.noreply.github.com> --- src/Common/isLocalAddress.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index 8089a713eb8..54a01dc4126 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -78,7 +78,7 @@ struct NetworkInterfaces freeifaddrs(ifaddr); } - static NetworkInterfaces & instance() + static const NetworkInterfaces & instance() { static constexpr int NET_INTERFACE_VALID_PERIOD_SECONDS = 30; static std::unique_ptr nf = std::make_unique(); From a175a7e0fcd696e08f3262582b5dbb7074259668 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 20 Sep 2023 09:04:35 +0000 Subject: [PATCH 40/69] Use error name instead of error code --- .../02494_query_cache_nondeterministic_functions.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql index 3a2e24d6bfe..62e0b099d7a 100644 --- a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql +++ b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql @@ -4,7 +4,7 @@ SYSTEM DROP QUERY CACHE; -- rand() is non-deterministic, the query is rejected by default -SELECT COUNT(rand(1)) SETTINGS use_query_cache = true; -- { serverError 703 } +SELECT COUNT(rand(1)) SETTINGS use_query_cache = true; -- { serverError CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS } SELECT COUNT(*) FROM system.query_cache; SELECT '---'; From eb8f9f1e764aa3b773a568f7234a989d55957891 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 20 Sep 2023 09:21:12 +0000 Subject: [PATCH 41/69] Remove config files sizes check --- .../internal/platform/data/file_test.go | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/programs/diagnostics/internal/platform/data/file_test.go b/programs/diagnostics/internal/platform/data/file_test.go index bb6397b82e8..9e305b1a5da 100644 --- a/programs/diagnostics/internal/platform/data/file_test.go +++ b/programs/diagnostics/internal/platform/data/file_test.go @@ -132,13 +132,6 @@ func TestConfigFileFrameCopy(t *testing.T) { configFrame, errs := data.NewConfigFileFrame(path.Join(cwd, "../../../testdata", "configs", "xml")) require.Empty(t, errs) i := 0 - sizes := map[string]int64{ - "users.xml": int64(2017), - "default-password.xml": int64(188), - "config.xml": int64(59377), - "server-include.xml": int64(168), - "user-include.xml": int64(559), - } var checkedFiles []string for { values, ok, err := configFrame.Next() @@ -153,8 +146,6 @@ func TestConfigFileFrameCopy(t *testing.T) { newPath := path.Join(tmrDir, fileName) err = configFile.Copy(newPath, true) require.FileExists(t, newPath) - destInfo, _ := os.Stat(newPath) - require.Equal(t, sizes[fileName], destInfo.Size()) require.Nil(t, err) bytes, err := ioutil.ReadFile(newPath) require.Nil(t, err) @@ -186,13 +177,6 @@ func TestConfigFileFrameCopy(t *testing.T) { configFrame, errs := data.NewConfigFileFrame(path.Join(cwd, "../../../testdata", "configs", "yaml")) require.Empty(t, errs) i := 0 - sizes := map[string]int64{ - "users.yaml": int64(1023), - "default-password.yaml": int64(132), - "config.yaml": int64(41633), - "server-include.yaml": int64(21), - "user-include.yaml": int64(120), - } var checkedFiles []string for { values, ok, err := configFrame.Next() @@ -207,8 +191,6 @@ func TestConfigFileFrameCopy(t *testing.T) { newPath := path.Join(tmrDir, fileName) err = configFile.Copy(newPath, true) require.FileExists(t, newPath) - destInfo, _ := os.Stat(newPath) - require.Equal(t, sizes[fileName], destInfo.Size()) require.Nil(t, err) bytes, err := ioutil.ReadFile(newPath) require.Nil(t, err) From 65baf129bf8a081d4041fa73aebb34287fcb098b Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 19 Sep 2023 15:34:42 +0000 Subject: [PATCH 42/69] Do not set PR status label --- tests/ci/commit_status_helper.py | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index a5fd27efb6b..3d07d81a1fe 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -141,16 +141,6 @@ STATUS_ICON_MAP = defaultdict( ) -def update_pr_status_label(pr: PullRequest, status: str) -> None: - new_label = "pr-status-" + STATUS_ICON_MAP[status] - for label in pr.get_labels(): - if label.name == new_label: - return - if label.name.startswith("pr-status-"): - pr.remove_from_labels(label.name) - pr.add_to_labels(new_label) - - def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: """It adds or updates the comment status to all Pull Requests but for release one, so the method does nothing for simple pushes and pull requests with @@ -190,8 +180,6 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: comment = ic break - update_pr_status_label(pr, get_worst_state(statuses)) - if comment is None: pr.create_issue_comment(comment_body) return From e6a3f3ed017a1f6ed82d00e210ed173c3da69242 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 20 Sep 2023 09:31:03 +0000 Subject: [PATCH 43/69] get rid of unique pointer Signed-off-by: Duc Canh Le --- src/Common/isLocalAddress.cpp | 26 +++++++++++++++++--------- 1 file changed, 17 insertions(+), 9 deletions(-) diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index 54a01dc4126..e9fefb0c319 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -1,12 +1,14 @@ #include #include +#include #include #include #include #include #include #include +#include #include #include #include @@ -23,7 +25,7 @@ namespace ErrorCodes namespace { -struct NetworkInterfaces +struct NetworkInterfaces : public boost::noncopyable { ifaddrs * ifaddr; NetworkInterfaces() @@ -34,6 +36,12 @@ struct NetworkInterfaces } } + void swap(NetworkInterfaces && other) + { + ifaddr = other.ifaddr; + other.ifaddr = nullptr; + } + bool hasAddress(const Poco::Net::IPAddress & address) const { ifaddrs * iface; @@ -80,24 +88,24 @@ struct NetworkInterfaces static const NetworkInterfaces & instance() { - static constexpr int NET_INTERFACE_VALID_PERIOD_SECONDS = 30; - static std::unique_ptr nf = std::make_unique(); - static time_t last_updated_time = time(nullptr); + static constexpr int NET_INTERFACE_VALID_PERIOD_MS = 30000; + static NetworkInterfaces nf; + static auto last_updated_time = std::chrono::steady_clock::now(); static std::shared_mutex nf_mtx; - time_t now = time(nullptr); + auto now = std::chrono::steady_clock::now(); - if (now - last_updated_time > NET_INTERFACE_VALID_PERIOD_SECONDS) + if (std::chrono::duration_cast(now - last_updated_time).count() > NET_INTERFACE_VALID_PERIOD_MS) { std::unique_lock lock(nf_mtx); - nf = std::make_unique(); + nf.swap(NetworkInterfaces()); last_updated_time = now; - return *nf; + return nf; } else { std::shared_lock lock(nf_mtx); - return *nf; + return nf; } } }; From d8adf05de25a61555ded0cdbec3dc2388d77909e Mon Sep 17 00:00:00 2001 From: SmitaRKulkarni <64093672+SmitaRKulkarni@users.noreply.github.com> Date: Wed, 20 Sep 2023 11:31:12 +0200 Subject: [PATCH 44/69] Added a new column _block_number (#47532) Added a new virtual column _block_number which is persisted on merges when allow_experimental_block_number_column is enabled --- .../mergetree-family/mergetree.md | 1 + .../settings/merge-tree-settings.md | 6 +++ src/Interpreters/InterpreterCreateQuery.cpp | 8 ++++ src/Interpreters/MutationsInterpreter.cpp | 26 +++++++++++- src/Interpreters/inplaceBlockConversions.cpp | 14 +++++-- src/Interpreters/inplaceBlockConversions.h | 2 +- .../Algorithms/SummingSortedAlgorithm.cpp | 7 ++++ src/Processors/Transforms/TTLTransform.cpp | 3 +- src/Storages/AlterCommands.cpp | 11 ++++- src/Storages/BlockNumberColumn.cpp | 23 +++++++++++ src/Storages/BlockNumberColumn.h | 16 ++++++++ src/Storages/ColumnsDescription.cpp | 6 +++ src/Storages/MergeTree/IMergeTreeDataPart.h | 4 -- src/Storages/MergeTree/IMergeTreeReader.cpp | 4 +- src/Storages/MergeTree/IMergeTreeReader.h | 2 +- src/Storages/MergeTree/MergeTask.cpp | 23 ++++++++++- src/Storages/MergeTree/MergeTask.h | 7 ++++ src/Storages/MergeTree/MergeTreeData.cpp | 4 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 7 ++++ .../MergeTreeDataPartWriterCompact.cpp | 12 +++++- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 12 +++++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 ++- .../MergeTree/MergeTreeSelectProcessor.cpp | 27 ++++++++++-- .../MergeTree/MergeTreeSequentialSource.cpp | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 3 +- src/Storages/StorageDistributed.cpp | 3 ++ src/Storages/StorageLog.cpp | 12 +++++- src/Storages/StorageSnapshot.cpp | 2 + .../02668_column_block_number.reference | 41 +++++++++++++++++++ .../0_stateless/02668_column_block_number.sql | 32 +++++++++++++++ ...lumn_block_number_vertical_merge.reference | 41 +++++++++++++++++++ ...668_column_block_number_vertical_merge.sql | 36 ++++++++++++++++ ...mn_block_number_with_projections.reference | 19 +++++++++ ...8_column_block_number_with_projections.sql | 18 ++++++++ 34 files changed, 411 insertions(+), 29 deletions(-) create mode 100644 src/Storages/BlockNumberColumn.cpp create mode 100644 src/Storages/BlockNumberColumn.h create mode 100644 tests/queries/0_stateless/02668_column_block_number.reference create mode 100644 tests/queries/0_stateless/02668_column_block_number.sql create mode 100644 tests/queries/0_stateless/02668_column_block_number_vertical_merge.reference create mode 100644 tests/queries/0_stateless/02668_column_block_number_vertical_merge.sql create mode 100644 tests/queries/0_stateless/02668_column_block_number_with_projections.reference create mode 100644 tests/queries/0_stateless/02668_column_block_number_with_projections.sql diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index bb257311680..23ab2699cc1 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1354,3 +1354,4 @@ In this sample configuration: - `_part_uuid` — Unique part identifier (if enabled MergeTree setting `assign_part_uuids`). - `_partition_value` — Values (a tuple) of a `partition by` expression. - `_sample_factor` — Sample factor (from the query). +- `_block_number` — Block number of the row, it is persisted on merges when `allow_experimental_block_number_column` is set to true. diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index e746719b629..483dcf2e61c 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -854,3 +854,9 @@ Possible values: - `Always` or `Never`. Default value: `Never` + +## allow_experimental_block_number_column + +Persists virtual column `_block_number` on merges. + +Default value: false. diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index b426025413c..3654f307eb9 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -31,6 +31,7 @@ #include #include #include +#include #include #include @@ -837,6 +838,13 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat "Cannot create table with column '{}' for *MergeTree engines because it " "is reserved for lightweight delete feature", LightweightDeleteDescription::FILTER_COLUMN.name); + + auto search_block_number = all_columns.find(BlockNumberColumn::name); + if (search_block_number != all_columns.end()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Cannot create table with column '{}' for *MergeTree engines because it " + "is reserved for storing block number", + BlockNumberColumn::name); } const auto & settings = getContext()->getSettingsRef(); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index e50f8488cac..4b0cbec4f9f 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -40,7 +41,6 @@ #include #include - namespace DB { @@ -56,6 +56,7 @@ namespace ErrorCodes extern const int THERE_IS_NO_COLUMN; } + namespace { @@ -416,6 +417,12 @@ static void validateUpdateColumns( found = true; } + /// Dont allow to override value of block number virtual column + if (!found && column_name == BlockNumberColumn::name) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Update is not supported for virtual column {} ", backQuote(column_name)); + } + if (!found) { for (const auto & col : metadata_snapshot->getColumns().getMaterialized()) @@ -511,7 +518,8 @@ void MutationsInterpreter::prepare(bool dry_run) for (const auto & [name, _] : command.column_to_update_expression) { - if (!available_columns_set.contains(name) && name != LightweightDeleteDescription::FILTER_COLUMN.name) + if (!available_columns_set.contains(name) && name != LightweightDeleteDescription::FILTER_COLUMN.name + && name != BlockNumberColumn::name) throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Column {} is updated but not requested to read", name); @@ -613,6 +621,8 @@ void MutationsInterpreter::prepare(bool dry_run) type = physical_column->type; else if (column == LightweightDeleteDescription::FILTER_COLUMN.name) type = LightweightDeleteDescription::FILTER_COLUMN.type; + else if (column == BlockNumberColumn::name) + type = BlockNumberColumn::type; else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown column {}", column); @@ -1087,6 +1097,18 @@ struct VirtualColumns virtuals.emplace_back(ColumnAndPosition{.column = std::move(column), .position = i}); } + else if (columns_to_read[i] == BlockNumberColumn::name) + { + if (!part->getColumns().contains(BlockNumberColumn::name)) + { + ColumnWithTypeAndName block_number_column; + block_number_column.type = BlockNumberColumn::type; + block_number_column.column = block_number_column.type->createColumnConst(0, part->info.min_block); + block_number_column.name = std::move(columns_to_read[i]); + + virtuals.emplace_back(ColumnAndPosition{.column = std::move(block_number_column), .position = i}); + } + } } if (!virtuals.empty()) diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 4cac2f0e20c..b7ef5dbdbbc 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -20,6 +20,7 @@ #include #include #include +#include namespace DB @@ -260,7 +261,7 @@ void fillMissingColumns( const NamesAndTypesList & requested_columns, const NamesAndTypesList & available_columns, const NameSet & partially_read_columns, - StorageMetadataPtr metadata_snapshot) + StorageMetadataPtr metadata_snapshot, size_t block_number) { size_t num_columns = requested_columns.size(); if (num_columns != res_columns.size()) @@ -339,9 +340,14 @@ void fillMissingColumns( } else { - /// We must turn a constant column into a full column because the interpreter could infer - /// that it is constant everywhere but in some blocks (from other parts) it can be a full column. - res_columns[i] = type->createColumnConstWithDefaultValue(num_rows)->convertToFullColumnIfConst(); + if (requested_column->name == BlockNumberColumn::name) + res_columns[i] = type->createColumnConst(num_rows, block_number)->convertToFullColumnIfConst(); + else + /// We must turn a constant column into a full column because the interpreter could infer + /// that it is constant everywhere but in some blocks (from other parts) it can be a full column. + res_columns[i] = type->createColumnConstWithDefaultValue(num_rows)->convertToFullColumnIfConst(); + + } } } diff --git a/src/Interpreters/inplaceBlockConversions.h b/src/Interpreters/inplaceBlockConversions.h index bea44bf6db9..7a13a75ec8b 100644 --- a/src/Interpreters/inplaceBlockConversions.h +++ b/src/Interpreters/inplaceBlockConversions.h @@ -46,6 +46,6 @@ void fillMissingColumns( const NamesAndTypesList & requested_columns, const NamesAndTypesList & available_columns, const NameSet & partially_read_columns, - StorageMetadataPtr metadata_snapshot); + StorageMetadataPtr metadata_snapshot, size_t block_number = 0); } diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index 7dac5715f95..f468e112d8f 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB @@ -222,6 +223,12 @@ static SummingSortedAlgorithm::ColumnsDefinition defineColumns( const ColumnWithTypeAndName & column = header.safeGetByPosition(i); const auto * simple = dynamic_cast(column.type->getCustomName()); + if (column.name == BlockNumberColumn::name) + { + def.column_numbers_not_to_aggregate.push_back(i); + continue; + } + /// Discover nested Maps and find columns for summation if (typeid_cast(column.type.get()) && !simple) { diff --git a/src/Processors/Transforms/TTLTransform.cpp b/src/Processors/Transforms/TTLTransform.cpp index 3250d012d5c..7cde86098c7 100644 --- a/src/Processors/Transforms/TTLTransform.cpp +++ b/src/Processors/Transforms/TTLTransform.cpp @@ -49,7 +49,8 @@ TTLTransform::TTLTransform( for (const auto & group_by_ttl : metadata_snapshot_->getGroupByTTLs()) algorithms.emplace_back(std::make_unique( - group_by_ttl, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_, getInputPort().getHeader(), storage_)); + group_by_ttl, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_, + getInputPort().getHeader(), storage_)); if (metadata_snapshot_->hasAnyColumnTTL()) { diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index f38fc1f3734..da46cb4d7fe 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -782,7 +783,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada /// Drop alias is metadata alter, in other case mutation is required. if (type == DROP_COLUMN) return metadata.columns.hasColumnOrNested(GetColumnsOptions::AllPhysical, column_name) || - column_name == LightweightDeleteDescription::FILTER_COLUMN.name; + column_name == LightweightDeleteDescription::FILTER_COLUMN.name || column_name == BlockNumberColumn::name; if (type != MODIFY_COLUMN || data_type == nullptr) return false; @@ -1066,6 +1067,10 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add column {}: " "this column name is reserved for lightweight delete feature", backQuote(column_name)); + if (column_name == BlockNumberColumn::name && std::dynamic_pointer_cast(table)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add column {}: " + "this column name is reserved for _block_number persisting feature", backQuote(column_name)); + if (command.codec) CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_deflate_qpl_codec); @@ -1270,6 +1275,10 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot rename to {}: " "this column name is reserved for lightweight delete feature", backQuote(command.rename_to)); + if (command.rename_to == BlockNumberColumn::name && std::dynamic_pointer_cast(table)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot rename to {}: " + "this column name is reserved for _block_number persisting feature", backQuote(command.rename_to)); + if (modified_columns.contains(column_name)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot rename and modify the same column {} " "in a single ALTER query", backQuote(column_name)); diff --git a/src/Storages/BlockNumberColumn.cpp b/src/Storages/BlockNumberColumn.cpp new file mode 100644 index 00000000000..8c9e1fd902a --- /dev/null +++ b/src/Storages/BlockNumberColumn.cpp @@ -0,0 +1,23 @@ +#include +#include + +namespace DB +{ + +CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); + +CompressionCodecPtr getCompressionCodecForBlockNumberColumn() +{ + std::vector codecs; + codecs.reserve(2); + auto data_bytes_size = BlockNumberColumn::type->getSizeOfValueInMemory(); + codecs.emplace_back(getCompressionCodecDelta(data_bytes_size)); + codecs.emplace_back(CompressionCodecFactory::instance().get("LZ4", {})); + return std::make_shared(codecs); +} + +const String BlockNumberColumn::name = "_block_number"; +const DataTypePtr BlockNumberColumn::type = std::make_shared(); +const CompressionCodecPtr BlockNumberColumn::compression_codec = getCompressionCodecForBlockNumberColumn(); + +} diff --git a/src/Storages/BlockNumberColumn.h b/src/Storages/BlockNumberColumn.h new file mode 100644 index 00000000000..fffa68bfd49 --- /dev/null +++ b/src/Storages/BlockNumberColumn.h @@ -0,0 +1,16 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +struct BlockNumberColumn +{ + static const String name; + static const DataTypePtr type; + static const CompressionCodecPtr compression_codec; +}; + +} diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 0c918bda5fd..9fa79387d5c 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -30,11 +30,15 @@ #include #include #include +#include namespace DB { +CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); + + namespace ErrorCodes { extern const int NO_SUCH_COLUMN_IN_TABLE; @@ -721,11 +725,13 @@ CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_ CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_name) const { + assert (column_name != BlockNumberColumn::name); return getCodecOrDefault(column_name, CompressionCodecFactory::instance().getDefaultCodec()); } ASTPtr ColumnsDescription::getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const { + assert (column_name != BlockNumberColumn::name); const auto it = columns.get<1>().find(column_name); if (it == columns.get<1>().end() || !it->codec) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 97c9b81ce87..02c838458f9 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -477,10 +477,6 @@ public: /// Moar hardening: this method is supposed to be used for debug assertions bool assertHasValidVersionMetadata() const; - /// Return hardlink count for part. - /// Required for keep data on remote FS when part has shadow copies. - UInt32 getNumberOfRefereneces() const; - /// True if the part supports lightweight delete mutate. bool supportLightweightDeleteMutate() const; diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 4bb8c400691..120edd81e30 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -62,7 +62,7 @@ const IMergeTreeReader::ValueSizeMap & IMergeTreeReader::getAvgValueSizeHints() return avg_value_size_hints; } -void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows) const +void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows, size_t block_number) const { try { @@ -71,7 +71,7 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e res_columns, num_rows, Nested::convertToSubcolumns(requested_columns), Nested::convertToSubcolumns(available_columns), - partially_read_columns, storage_snapshot->metadata); + partially_read_columns, storage_snapshot->metadata, block_number); should_evaluate_missing_defaults = std::any_of( res_columns.begin(), res_columns.end(), [](const auto & column) { return column == nullptr; }); diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index a7e60254217..02faebf4b41 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -45,7 +45,7 @@ public: /// Add columns from ordered_names that are not present in the block. /// Missing columns are added in the order specified by ordered_names. /// num_rows is needed in case if all res_columns are nullptr. - void fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows) const; + void fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows, size_t block_number = 0) const; /// Evaluate defaulted columns if necessary. void evaluateMissingDefaults(Block additional_columns, Columns & res_columns) const; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index df607d36402..adb1ca72e46 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -218,6 +218,14 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() ctx->need_remove_expired_values = false; ctx->force_ttl = false; + if (supportsBlockNumberColumn(global_ctx) && !global_ctx->storage_columns.contains(BlockNumberColumn::name)) + { + global_ctx->storage_columns.emplace_back(NameAndTypePair{BlockNumberColumn::name,BlockNumberColumn::type}); + global_ctx->all_column_names.emplace_back(BlockNumberColumn::name); + global_ctx->gathering_columns.emplace_back(NameAndTypePair{BlockNumberColumn::name,BlockNumberColumn::type}); + global_ctx->gathering_column_names.emplace_back(BlockNumberColumn::name); + } + SerializationInfo::Settings info_settings = { .ratio_of_defaults_for_sparse = global_ctx->data->getSettings()->ratio_of_defaults_for_sparse_serialization, @@ -251,12 +259,12 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() } } - global_ctx->new_data_part->setColumns(global_ctx->storage_columns, infos, global_ctx->metadata_snapshot->getMetadataVersion()); - const auto & local_part_min_ttl = global_ctx->new_data_part->ttl_infos.part_min_ttl; if (local_part_min_ttl && local_part_min_ttl <= global_ctx->time_of_merge) ctx->need_remove_expired_values = true; + global_ctx->new_data_part->setColumns(global_ctx->storage_columns, infos, global_ctx->metadata_snapshot->getMetadataVersion()); + if (ctx->need_remove_expired_values && global_ctx->ttl_merges_blocker->isCancelled()) { LOG_INFO(ctx->log, "Part {} has values with expired TTL, but merges with TTL are cancelled.", global_ctx->new_data_part->name); @@ -998,6 +1006,17 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() if (global_ctx->deduplicate) { + /// We don't want to deduplicate by block number column + /// so if deduplicate_by_columns is empty, add all columns except _block_number + if (supportsBlockNumberColumn(global_ctx) && global_ctx->deduplicate_by_columns.empty()) + { + for (const auto & col : global_ctx->merging_column_names) + { + if (col != BlockNumberColumn::name) + global_ctx->deduplicate_by_columns.emplace_back(col); + } + } + if (DistinctSortedTransform::isApplicable(header, sort_description, global_ctx->deduplicate_by_columns)) res_pipe.addTransform(std::make_shared( res_pipe.getHeader(), sort_description, SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns)); diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 402d3c26e49..2122484bb24 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -388,6 +389,12 @@ private: Stages::iterator stages_iterator = stages.begin(); + /// Check for persisting block number column + static bool supportsBlockNumberColumn(GlobalRuntimeContextPtr global_ctx) + { + return global_ctx->data->getSettings()->allow_experimental_block_number_column && global_ctx->metadata_snapshot->getGroupByTTLs().empty(); + } + }; /// FIXME diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4b6d2ea41ed..14c9961f6c3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -78,6 +78,7 @@ #include #include #include +#include #include #include @@ -3730,7 +3731,7 @@ void MergeTreeData::checkPartDynamicColumns(MutableDataPartPtr & part, DataParts const auto & part_columns = part->getColumns(); for (const auto & part_column : part_columns) { - if (part_column.name == LightweightDeleteDescription::FILTER_COLUMN.name) + if (part_column.name == LightweightDeleteDescription::FILTER_COLUMN.name || part_column.name == BlockNumberColumn::name) continue; auto storage_column = columns.getPhysical(part_column.name); @@ -8269,6 +8270,7 @@ NamesAndTypesList MergeTreeData::getVirtuals() const NameAndTypePair("_sample_factor", std::make_shared()), NameAndTypePair("_part_offset", std::make_shared()), LightweightDeleteDescription::FILTER_COLUMN, + NameAndTypePair(BlockNumberColumn::name, BlockNumberColumn::type), }; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index c6d059498ff..209a5061f33 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -64,6 +65,12 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( ordered_columns_list.sort([this](const auto & lhs, const auto & rhs) { return *getColumnPosition(lhs.name) < *getColumnPosition(rhs.name); }); + /// _block_number column is not added by user, but is persisted in a part after merge + /// If _block_number is not present in the parts to be merged, then it won't have a position + /// So check if its not present and add it at the end + if (columns_list.contains(BlockNumberColumn::name) && !ordered_columns_list.contains(BlockNumberColumn::name)) + ordered_columns_list.emplace_back(NameAndTypePair{BlockNumberColumn::name, BlockNumberColumn::type}); + return std::make_unique( shared_from_this(), ordered_columns_list, metadata_snapshot, indices_to_recalc, getMarksFileExtension(), diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 3475130bf24..d2a9632d4e5 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -1,9 +1,12 @@ #include #include +#include namespace DB { + CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -53,7 +56,14 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const auto & storage_columns = metadata_snapshot->getColumns(); for (const auto & column : columns_list) - addStreams(column, storage_columns.getCodecDescOrDefault(column.name, default_codec)); + { + ASTPtr compression; + if (column.name == BlockNumberColumn::name) + compression = BlockNumberColumn::compression_codec->getFullCodecDesc(); + else + compression = storage_columns.getCodecDescOrDefault(column.name, default_codec); + addStreams(column, compression); + } } void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, const ASTPtr & effective_codec_desc) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index bcf340e0f55..f3e60e224aa 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -6,9 +6,12 @@ #include #include #include +#include namespace DB { + CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -87,7 +90,14 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( { const auto & columns = metadata_snapshot->getColumns(); for (const auto & it : columns_list) - addStreams(it, columns.getCodecDescOrDefault(it.name, default_codec)); + { + ASTPtr compression; + if (it.name == BlockNumberColumn::name) + compression = BlockNumberColumn::compression_codec->getFullCodecDesc(); + else + compression = columns.getCodecDescOrDefault(it.name, default_codec); + addStreams(it, compression); + } } void MergeTreeDataPartWriterWide::addStreams( diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 3c2b09b6f3b..d8ebfb95d74 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -46,7 +46,7 @@ #include #include - +#include #include namespace CurrentMetrics @@ -1232,6 +1232,10 @@ static void selectColumnNames( { virt_column_names.push_back(name); } + else if (name == BlockNumberColumn::name) + { + virt_column_names.push_back(name); + } else if (name == "_part_uuid") { virt_column_names.push_back(name); diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 95fcde23f8e..9b480ac27a0 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include namespace DB @@ -24,7 +25,8 @@ namespace ErrorCodes static void injectNonConstVirtualColumns( size_t rows, Block & block, - const Names & virtual_columns); + const Names & virtual_columns, + MergeTreeReadTask * task = nullptr); static void injectPartConstVirtualColumns( size_t rows, @@ -247,7 +249,8 @@ namespace static void injectNonConstVirtualColumns( size_t rows, Block & block, - const Names & virtual_columns) + const Names & virtual_columns, + MergeTreeReadTask * task) { VirtualColumnsInserter inserter(block); for (const auto & virtual_column_name : virtual_columns) @@ -278,6 +281,24 @@ static void injectNonConstVirtualColumns( inserter.insertUInt8Column(column, virtual_column_name); } + + if (virtual_column_name == BlockNumberColumn::name) + { + ColumnPtr column; + if (rows) + { + size_t value = 0; + if (task) + { + value = task->getInfo().data_part ? task->getInfo().data_part->info.min_block : 0; + } + column = BlockNumberColumn::type->createColumnConst(rows, value)->convertToFullColumnIfConst(); + } + else + column = BlockNumberColumn::type->createColumn(); + + inserter.insertUInt64Column(column, virtual_column_name); + } } } @@ -368,7 +389,7 @@ void MergeTreeSelectProcessor::injectVirtualColumns( { /// First add non-const columns that are filled by the range reader and then const columns that we will fill ourselves. /// Note that the order is important: virtual columns filled by the range reader must go first - injectNonConstVirtualColumns(row_count, block, virtual_columns); + injectNonConstVirtualColumns(row_count, block, virtual_columns,task); injectPartConstVirtualColumns(row_count, block, task, partition_value_type, virtual_columns); } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 8a9faa5cee4..6ba0d4dcfad 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -176,7 +176,7 @@ try current_mark += (rows_to_read == rows_read); bool should_evaluate_missing_defaults = false; - reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_read); + reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_read, data_part->info.min_block); if (should_evaluate_missing_defaults) { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 4f36da048c2..52c3f3efc6d 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -171,7 +171,8 @@ struct Settings; M(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ M(Bool, allow_remote_fs_zero_copy_replication, false, "Don't use this setting in production, because it is not ready.", 0) \ M(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for zero-copy table-independent info.", 0) \ - M(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", 0) \ + M(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", 0) \ + M(Bool, allow_experimental_block_number_column, false, "Enable persisting column _block_number for each row.", 0) \ \ /** Compress marks and primary key. */ \ M(Bool, compress_marks, true, "Marks support compression, reduce mark file size and speed up network transmission.", 0) \ diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f869a1158ef..4baa43311d4 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -102,6 +102,8 @@ #include #include +#include + #include #include #include @@ -298,6 +300,7 @@ NamesAndTypesList StorageDistributed::getVirtuals() const NameAndTypePair("_sample_factor", std::make_shared()), NameAndTypePair("_part_offset", std::make_shared()), NameAndTypePair("_row_exists", std::make_shared()), + NameAndTypePair(BlockNumberColumn::name, BlockNumberColumn::type), NameAndTypePair("_shard_num", std::make_shared()), /// deprecated }; } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index ab795a2d04c..cf76f7a16ba 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include #include @@ -45,6 +46,8 @@ namespace DB { + CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size); + namespace ErrorCodes { extern const int TIMEOUT_EXCEEDED; @@ -452,10 +455,15 @@ void LogSink::writeData(const NameAndTypePair & name_and_type, const IColumn & c const auto & data_file = *data_file_it->second; const auto & columns = metadata_snapshot->getColumns(); + CompressionCodecPtr compression; + if (name_and_type.name == BlockNumberColumn::name) + compression = BlockNumberColumn::compression_codec; + else + compression = columns.getCodecOrDefault(name_and_type.name); + it = streams.try_emplace(data_file.name, storage.disk, data_file.path, storage.file_checker.getFileSize(data_file.path), - columns.getCodecOrDefault(name_and_type.name), - storage.max_compress_block_size).first; + compression, storage.max_compress_block_size).first; } auto & stream = it->second; diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index c0e85900794..0c19634f50c 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -24,6 +25,7 @@ void StorageSnapshot::init() if (storage.hasLightweightDeletedMask()) system_columns[LightweightDeleteDescription::FILTER_COLUMN.name] = LightweightDeleteDescription::FILTER_COLUMN.type; + system_columns[BlockNumberColumn::name] = BlockNumberColumn::type; } NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options) const diff --git a/tests/queries/0_stateless/02668_column_block_number.reference b/tests/queries/0_stateless/02668_column_block_number.reference new file mode 100644 index 00000000000..f08220243ff --- /dev/null +++ b/tests/queries/0_stateless/02668_column_block_number.reference @@ -0,0 +1,41 @@ +*** BEFORE MUTATION BEFORE MERGE *** +1 1 1 all_1_1_0 +2 2 1 all_1_1_0 +3 3 1 all_1_1_0 +4 4 2 all_2_2_0 +5 5 2 all_2_2_0 +6 6 2 all_2_2_0 +*** AFTER MUTATION BEFORE MERGE *** +1 0 1 all_1_1_0_3 +2 0 1 all_1_1_0_3 +3 0 1 all_1_1_0_3 +4 4 2 all_2_2_0_3 +5 5 2 all_2_2_0_3 +6 6 2 all_2_2_0_3 +*** AFTER MUTATION AFTER MERGE *** +1 0 1 all_1_2_1_3 +2 0 1 all_1_2_1_3 +3 0 1 all_1_2_1_3 +4 4 2 all_1_2_1_3 +5 5 2 all_1_2_1_3 +6 6 2 all_1_2_1_3 +*** AFTER MUTATION AFTER MERGE , NEW BLOCK *** +1 0 1 all_1_2_1_3 +2 0 1 all_1_2_1_3 +3 0 1 all_1_2_1_3 +4 4 2 all_1_2_1_3 +5 5 2 all_1_2_1_3 +6 6 2 all_1_2_1_3 +7 7 4 all_4_4_0 +8 8 4 all_4_4_0 +9 9 4 all_4_4_0 +*** AFTER MUTATION AFTER MERGE , NEW BLOCK MERGED *** +1 0 1 all_1_4_2_3 +2 0 1 all_1_4_2_3 +3 0 1 all_1_4_2_3 +4 4 2 all_1_4_2_3 +5 5 2 all_1_4_2_3 +6 6 2 all_1_4_2_3 +7 7 4 all_1_4_2_3 +8 8 4 all_1_4_2_3 +9 9 4 all_1_4_2_3 diff --git a/tests/queries/0_stateless/02668_column_block_number.sql b/tests/queries/0_stateless/02668_column_block_number.sql new file mode 100644 index 00000000000..4d4d54bba5f --- /dev/null +++ b/tests/queries/0_stateless/02668_column_block_number.sql @@ -0,0 +1,32 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test (id UInt32, a UInt32) ENGINE = MergeTree ORDER BY id SETTINGS allow_experimental_block_number_column = true; + +INSERT INTO test(id,a) VALUES (1,1),(2,2),(3,3); +INSERT INTO test(id,a) VALUES (4,4),(5,5),(6,6); + +SELECT '*** BEFORE MUTATION BEFORE MERGE ***'; +SELECT id,a,_block_number,_part from test ORDER BY id; + +set mutations_sync=1; +ALTER TABLE test UPDATE a=0 WHERE id<4; + +SELECT '*** AFTER MUTATION BEFORE MERGE ***'; +SELECT id,a,_block_number,_part from test ORDER BY id; + +OPTIMIZE TABLE test FINAL; + +SELECT '*** AFTER MUTATION AFTER MERGE ***'; +SELECT *,_block_number,_part from test ORDER BY id; + +INSERT INTO test(id,a) VALUES (7,7),(8,8),(9,9); + +SELECT '*** AFTER MUTATION AFTER MERGE , NEW BLOCK ***'; +SELECT *,_block_number,_part from test ORDER BY id; + +OPTIMIZE TABLE test FINAL; + +SELECT '*** AFTER MUTATION AFTER MERGE , NEW BLOCK MERGED ***'; +SELECT *,_block_number,_part from test ORDER BY id; + +DROP TABLE test; \ No newline at end of file diff --git a/tests/queries/0_stateless/02668_column_block_number_vertical_merge.reference b/tests/queries/0_stateless/02668_column_block_number_vertical_merge.reference new file mode 100644 index 00000000000..f08220243ff --- /dev/null +++ b/tests/queries/0_stateless/02668_column_block_number_vertical_merge.reference @@ -0,0 +1,41 @@ +*** BEFORE MUTATION BEFORE MERGE *** +1 1 1 all_1_1_0 +2 2 1 all_1_1_0 +3 3 1 all_1_1_0 +4 4 2 all_2_2_0 +5 5 2 all_2_2_0 +6 6 2 all_2_2_0 +*** AFTER MUTATION BEFORE MERGE *** +1 0 1 all_1_1_0_3 +2 0 1 all_1_1_0_3 +3 0 1 all_1_1_0_3 +4 4 2 all_2_2_0_3 +5 5 2 all_2_2_0_3 +6 6 2 all_2_2_0_3 +*** AFTER MUTATION AFTER MERGE *** +1 0 1 all_1_2_1_3 +2 0 1 all_1_2_1_3 +3 0 1 all_1_2_1_3 +4 4 2 all_1_2_1_3 +5 5 2 all_1_2_1_3 +6 6 2 all_1_2_1_3 +*** AFTER MUTATION AFTER MERGE , NEW BLOCK *** +1 0 1 all_1_2_1_3 +2 0 1 all_1_2_1_3 +3 0 1 all_1_2_1_3 +4 4 2 all_1_2_1_3 +5 5 2 all_1_2_1_3 +6 6 2 all_1_2_1_3 +7 7 4 all_4_4_0 +8 8 4 all_4_4_0 +9 9 4 all_4_4_0 +*** AFTER MUTATION AFTER MERGE , NEW BLOCK MERGED *** +1 0 1 all_1_4_2_3 +2 0 1 all_1_4_2_3 +3 0 1 all_1_4_2_3 +4 4 2 all_1_4_2_3 +5 5 2 all_1_4_2_3 +6 6 2 all_1_4_2_3 +7 7 4 all_1_4_2_3 +8 8 4 all_1_4_2_3 +9 9 4 all_1_4_2_3 diff --git a/tests/queries/0_stateless/02668_column_block_number_vertical_merge.sql b/tests/queries/0_stateless/02668_column_block_number_vertical_merge.sql new file mode 100644 index 00000000000..a32060efae1 --- /dev/null +++ b/tests/queries/0_stateless/02668_column_block_number_vertical_merge.sql @@ -0,0 +1,36 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test (id UInt32, a UInt32) ENGINE = MergeTree ORDER BY id SETTINGS allow_experimental_block_number_column = true, + vertical_merge_algorithm_min_rows_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 0, + min_rows_for_wide_part = 1, + min_bytes_for_wide_part = 1; + +INSERT INTO test(id,a) VALUES (1,1),(2,2),(3,3); +INSERT INTO test(id,a) VALUES (4,4),(5,5),(6,6); + +SELECT '*** BEFORE MUTATION BEFORE MERGE ***'; +SELECT id,a,_block_number,_part from test ORDER BY id; + +set mutations_sync=1; +ALTER TABLE test UPDATE a=0 WHERE id<4; + +SELECT '*** AFTER MUTATION BEFORE MERGE ***'; +SELECT id,a,_block_number,_part from test ORDER BY id; + +OPTIMIZE TABLE test FINAL; + +SELECT '*** AFTER MUTATION AFTER MERGE ***'; +SELECT *,_block_number,_part from test ORDER BY id; + +INSERT INTO test(id,a) VALUES (7,7),(8,8),(9,9); + +SELECT '*** AFTER MUTATION AFTER MERGE , NEW BLOCK ***'; +SELECT *,_block_number,_part from test ORDER BY id; + +OPTIMIZE TABLE test FINAL; + +SELECT '*** AFTER MUTATION AFTER MERGE , NEW BLOCK MERGED ***'; +SELECT *,_block_number,_part from test ORDER BY id; + +DROP TABLE test; \ No newline at end of file diff --git a/tests/queries/0_stateless/02668_column_block_number_with_projections.reference b/tests/queries/0_stateless/02668_column_block_number_with_projections.reference new file mode 100644 index 00000000000..289a21035b5 --- /dev/null +++ b/tests/queries/0_stateless/02668_column_block_number_with_projections.reference @@ -0,0 +1,19 @@ +0 +1 +2 +3 +*** AFTER FIRST OPTIMIZE *** +0 1 +1 2 +1 2 +2 3 +3 3 +*** AFTER SECOND OPTIMIZE *** +0 1 +1 2 +1 2 +2 3 +3 3 +4 4 +5 4 +6 4 diff --git a/tests/queries/0_stateless/02668_column_block_number_with_projections.sql b/tests/queries/0_stateless/02668_column_block_number_with_projections.sql new file mode 100644 index 00000000000..25ec411967b --- /dev/null +++ b/tests/queries/0_stateless/02668_column_block_number_with_projections.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS t; +CREATE TABLE t (x UInt8, PROJECTION p (SELECT x GROUP BY x)) ENGINE = MergeTree ORDER BY () SETTINGS allow_experimental_block_number_column=true; +INSERT INTO t VALUES (0); +INSERT INTO t VALUES (1),(1); +INSERT INTO t VALUES (2),(3); + +SELECT x FROM t GROUP BY x; +OPTIMIZE TABLE t FINAL; + +SELECT '*** AFTER FIRST OPTIMIZE ***'; +SELECT x,_block_number FROM t; + +INSERT INTO t VALUES (4), (5), (6); +OPTIMIZE TABLE t FINAL; +SELECT '*** AFTER SECOND OPTIMIZE ***'; +SELECT x,_block_number FROM t; + +DROP TABLE t; \ No newline at end of file From 1200e03f207e17d175fb53c10846bf80ac9c707d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 20 Sep 2023 18:51:31 +0800 Subject: [PATCH 45/69] correctly implement swap --- src/Common/isLocalAddress.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index e9fefb0c319..ce0316db2c4 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -38,8 +38,9 @@ struct NetworkInterfaces : public boost::noncopyable void swap(NetworkInterfaces && other) { + auto * tmp = ifaddr; ifaddr = other.ifaddr; - other.ifaddr = nullptr; + other.ifaddr = tmp; } bool hasAddress(const Poco::Net::IPAddress & address) const From 2805ebf2b259bf99382ebdc537139ff8e6a3973a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 20 Sep 2023 11:08:02 +0000 Subject: [PATCH 46/69] Set correct size for signal pipe buffer --- src/Daemon/BaseDaemon.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index be323dc6786..8e01311dcb0 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -104,7 +104,8 @@ static const size_t signal_pipe_buf_size = + sizeof(ucontext_t*) + sizeof(StackTrace) + sizeof(UInt32) - + sizeof(void*); + + sizeof(void*) + + sizeof(UInt64); using signal_function = void(int, siginfo_t*, void*); From 06513f60ebf548f69f47c4dde649ff30bea6b971 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 20 Sep 2023 15:16:39 +0200 Subject: [PATCH 47/69] Revert "refine error code of duplicated index in create query" --- src/Interpreters/InterpreterCreateQuery.cpp | 4 ---- .../0_stateless/02884_duplicate_index_name.reference | 0 .../queries/0_stateless/02884_duplicate_index_name.sql | 10 ---------- 3 files changed, 14 deletions(-) delete mode 100644 tests/queries/0_stateless/02884_duplicate_index_name.reference delete mode 100644 tests/queries/0_stateless/02884_duplicate_index_name.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 3654f307eb9..a0635f18214 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -96,7 +96,6 @@ namespace ErrorCodes extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY; extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE; extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_INDEX; extern const int LOGICAL_ERROR; extern const int UNKNOWN_DATABASE; extern const int PATH_ACCESS_DENIED; @@ -699,8 +698,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti for (const auto & index : create.columns_list->indices->children) { IndexDescription index_desc = IndexDescription::getIndexFromAST(index->clone(), properties.columns, getContext()); - if (properties.indices.has(index_desc.name)) - throw Exception(ErrorCodes::ILLEGAL_INDEX, "Duplicated index name {}", backQuoteIfNeed(index_desc.name)); const auto & settings = getContext()->getSettingsRef(); if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index) { @@ -715,7 +712,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti properties.indices.push_back(index_desc); } - if (create.columns_list->projections) for (const auto & projection_ast : create.columns_list->projections->children) { diff --git a/tests/queries/0_stateless/02884_duplicate_index_name.reference b/tests/queries/0_stateless/02884_duplicate_index_name.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02884_duplicate_index_name.sql b/tests/queries/0_stateless/02884_duplicate_index_name.sql deleted file mode 100644 index 4cd9ae6d2a2..00000000000 --- a/tests/queries/0_stateless/02884_duplicate_index_name.sql +++ /dev/null @@ -1,10 +0,0 @@ -DROP TABLE IF EXISTS test_dup_index; - -CREATE TABLE test_dup_index -( - a Int64, - b Int64, - INDEX idx_a a TYPE minmax, - INDEX idx_a b TYPE minmax -) Engine = MergeTree() -ORDER BY a; -- { serverError ILLEGAL_INDEX } From c75f7c843456fe184f1d7d5e40e77d27123a441b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 20 Sep 2023 13:27:11 +0000 Subject: [PATCH 48/69] Correct merge result --- src/Common/ErrorCodes.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 95614c7e9ce..be2b0a7bd5e 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -585,8 +585,9 @@ M(700, USER_SESSION_LIMIT_EXCEEDED) \ M(701, CLUSTER_DOESNT_EXIST) \ M(702, CLIENT_INFO_DOES_NOT_MATCH) \ - M(703, CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS) \ - M(704, INVALID_IDENTIFIER) \ + M(703, INVALID_IDENTIFIER) \ + M(704, CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS) \ + \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ From 1a9467535066f54dc0dcf0f8c0e75dcd6fb9509b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 20 Sep 2023 13:32:01 +0000 Subject: [PATCH 49/69] Don't capture this in callback --- src/Server/KeeperTCPHandler.cpp | 2 +- src/Server/KeeperTCPHandler.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 58d227a5ae5..84ed7388503 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -382,7 +382,7 @@ void KeeperTCPHandler::runImpl() } auto response_fd = poll_wrapper->getResponseFD(); - auto response_callback = [this, response_fd] (const Coordination::ZooKeeperResponsePtr & response) + auto response_callback = [responses = this->responses, response_fd](const Coordination::ZooKeeperResponsePtr & response) { if (!responses->push(response)) throw Exception(ErrorCodes::SYSTEM_ERROR, diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index ffdd50b805a..588cdf6305e 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -25,7 +25,7 @@ struct SocketInterruptablePollWrapper; using SocketInterruptablePollWrapperPtr = std::unique_ptr; using ThreadSafeResponseQueue = ConcurrentBoundedQueue; -using ThreadSafeResponseQueuePtr = std::unique_ptr; +using ThreadSafeResponseQueuePtr = std::shared_ptr; struct LastOp; using LastOpMultiVersion = MultiVersion; From 4808c34f3b84756b6f0a07befb68decc062b3268 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 20 Sep 2023 13:45:54 +0000 Subject: [PATCH 50/69] Fix log message --- src/Interpreters/ClusterProxy/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index b67208ab12a..6c37046c5f2 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -199,7 +199,7 @@ void executeQuery( { LOG_TRACE( log, - "Parallel reading from replicas is disabled for cluster. There are no shards with more then 1 replica: cluster={}", + "Parallel reading from replicas is disabled for cluster. There are no shards with more than 1 replica: cluster={}", cluster->getName()); } From c9bf365767597874de1d768835581b87ad858f7d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 20 Sep 2023 13:47:11 +0000 Subject: [PATCH 51/69] Polishing --- src/Interpreters/ClusterProxy/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 6c37046c5f2..84ae382487d 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -193,7 +193,7 @@ void executeQuery( auto cluster = query_info.getCluster(); auto new_context = updateSettingsForCluster(*cluster, context, settings, main_table, query_info.additional_filter_ast, log); - if (context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value + if (context->getSettingsRef().allow_experimental_parallel_reading_from_replicas && context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value != new_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value) { From a1c98bc8c9730a199c15da198198ccceef39d74f Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 20 Sep 2023 09:50:35 -0400 Subject: [PATCH 52/69] fix build --- src/Storages/S3Queue/S3QueueSource.cpp | 5 +++++ src/Storages/S3Queue/S3QueueSource.h | 2 ++ 2 files changed, 7 insertions(+) diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 54a863aeb2c..5f640239985 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -147,6 +147,11 @@ StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::QueueGlobIterator::next( return KeyWithInfo(); } +size_t StorageS3QueueSource::QueueGlobIterator::estimatedKeysCount() +{ + return keys_buf.size(); +} + StorageS3QueueSource::StorageS3QueueSource( const ReadFromFormatInfo & info, const String & format_, diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index f89384fb096..0f83ed054d5 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -54,6 +54,8 @@ public: Strings filterProcessingFiles(const S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file = ""); + size_t estimatedKeysCount() override; + private: UInt64 max_poll_size; KeysWithInfo keys_buf; From 6dab5bf3a7571055d887ebe84d37fbb021fac9b4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 20 Sep 2023 15:55:12 +0200 Subject: [PATCH 53/69] Better --- src/Daemon/BaseDaemon.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 8e01311dcb0..f64d4b365a9 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -103,9 +103,9 @@ static const size_t signal_pipe_buf_size = + sizeof(siginfo_t) + sizeof(ucontext_t*) + sizeof(StackTrace) + + sizeof(UInt64) + sizeof(UInt32) - + sizeof(void*) - + sizeof(UInt64); + + sizeof(void*); using signal_function = void(int, siginfo_t*, void*); From 729c8aa29f6436d338f544d923677737f09e0660 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 20 Sep 2023 10:41:47 -0400 Subject: [PATCH 54/69] fix glob iterator estimated objects --- src/Storages/StorageS3.cpp | 19 +++---------------- 1 file changed, 3 insertions(+), 16 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index e99be7a1204..288f5423c00 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -182,9 +182,7 @@ public: size_t objectsCount() { - assert(outcome_future.valid()); - first_outcome = outcome_future.get(); - return first_outcome->GetResult().GetContents().size(); + return buffer.size(); } ~Impl() @@ -231,18 +229,8 @@ private: void fillInternalBufferAssumeLocked() { buffer.clear(); - - ListObjectsOutcome outcome; - if (unlikely(first_outcome)) - { - outcome = std::move(*first_outcome); - first_outcome = std::nullopt; - } - else - { - assert(outcome_future.valid()); - outcome = outcome_future.get(); - } + assert(outcome_future.valid()); + auto outcome = outcome_future.get(); if (!outcome.IsSuccess()) { @@ -359,7 +347,6 @@ private: ThreadPool list_objects_pool; ThreadPoolCallbackRunner list_objects_scheduler; std::future outcome_future; - std::optional first_outcome; /// the result will be set by `estimatedKeysCount` std::function file_progress_callback; }; From 69b36b9c88a9a41c7936c317071b24fce6a47d08 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Wed, 20 Sep 2023 16:42:32 +0200 Subject: [PATCH 55/69] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 24a03e283e6..a6e23e92d36 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -31,7 +31,7 @@ * Add new functions `structureToCapnProtoSchema`/`structureToProtobufSchema` that convert ClickHouse table structure to CapnProto/Protobuf format schema. Allow to input/output data in CapnProto/Protobuf format without external format schema using autogenerated schema from table structure (controled by settings `format_capn_proto_use_autogenerated_schema`/`format_protobuf_use_autogenerated_schema`). Allow to export autogenerated schema while input/outoput using setting `output_format_schema`. [#52278](https://github.com/ClickHouse/ClickHouse/pull/52278) ([Kruglov Pavel](https://github.com/Avogar)). * A new field `query_cache_usage` in `system.query_log` now shows if and how the query cache was used. [#52384](https://github.com/ClickHouse/ClickHouse/pull/52384) ([Robert Schulze](https://github.com/rschu1ze)). * Add new function `startsWithUTF8` and `endsWithUTF8`. [#52555](https://github.com/ClickHouse/ClickHouse/pull/52555) ([李扬](https://github.com/taiyang-li)). -* Allow variable number of columns in TSV/CuatomSeprarated/JSONCompactEachRow, make schema inference work with variable number of columns. Add settings `input_format_tsv_allow_variable_number_of_columns`, `input_format_custom_allow_variable_number_of_columns`, `input_format_json_compact_allow_variable_number_of_columns`. [#52692](https://github.com/ClickHouse/ClickHouse/pull/52692) ([Kruglov Pavel](https://github.com/Avogar)). +* Allow variable number of columns in TSV/CustomSeprarated/JSONCompactEachRow, make schema inference work with variable number of columns. Add settings `input_format_tsv_allow_variable_number_of_columns`, `input_format_custom_allow_variable_number_of_columns`, `input_format_json_compact_allow_variable_number_of_columns`. [#52692](https://github.com/ClickHouse/ClickHouse/pull/52692) ([Kruglov Pavel](https://github.com/Avogar)). * Added `SYSTEM STOP/START PULLING REPLICATION LOG` queries (for testing `ReplicatedMergeTree`). [#52881](https://github.com/ClickHouse/ClickHouse/pull/52881) ([Alexander Tokmakov](https://github.com/tavplubix)). * Allow to execute constant non-deterministic functions in mutations on initiator. [#53129](https://github.com/ClickHouse/ClickHouse/pull/53129) ([Anton Popov](https://github.com/CurtizJ)). * Add input format `One` that doesn't read any data and always returns single row with column `dummy` with type `UInt8` and value `0` like `system.one`. It can be used together with `_file/_path` virtual columns to list files in file/s3/url/hdfs/etc table functions without reading any data. [#53209](https://github.com/ClickHouse/ClickHouse/pull/53209) ([Kruglov Pavel](https://github.com/Avogar)). From c706101891dc491fab08de3a62d959e2fd19d8e4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 11 Aug 2023 14:29:37 +0200 Subject: [PATCH 56/69] Fix throttling of BACKUPs from/to S3 (in case native copy was not used) In some cases native copy is not possible, and such requests should be throttled. v0: copyS3FileNativeWithFallback v2: revert v0 and pass write_settings v3: pass read_settings to copyFile() Signed-off-by: Azat Khuzhin --- programs/disks/CommandCopy.cpp | 2 +- src/Backups/BackupIO_Disk.cpp | 4 +- src/Backups/BackupIO_S3.cpp | 2 + src/Disks/DiskEncrypted.cpp | 6 +-- src/Disks/DiskEncrypted.h | 2 +- src/Disks/DiskEncryptedTransaction.cpp | 4 +- src/Disks/DiskEncryptedTransaction.h | 2 +- src/Disks/DiskLocal.cpp | 6 +-- src/Disks/DiskLocal.h | 2 +- src/Disks/FakeDiskTransaction.h | 4 +- src/Disks/IDisk.cpp | 24 +++++------ src/Disks/IDisk.h | 7 ++-- src/Disks/IDiskTransaction.h | 6 ++- .../AzureBlobStorage/AzureObjectStorage.cpp | 2 + .../AzureBlobStorage/AzureObjectStorage.h | 2 + .../Cached/CachedObjectStorage.cpp | 12 ++++-- .../Cached/CachedObjectStorage.h | 4 ++ .../ObjectStorages/DiskObjectStorage.cpp | 7 ++-- src/Disks/ObjectStorages/DiskObjectStorage.h | 3 +- ...jectStorageRemoteMetadataRestoreHelper.cpp | 6 +-- ...ObjectStorageRemoteMetadataRestoreHelper.h | 4 +- .../DiskObjectStorageTransaction.cpp | 17 +++++--- .../DiskObjectStorageTransaction.h | 2 +- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 6 ++- .../ObjectStorages/HDFS/HDFSObjectStorage.h | 2 + src/Disks/ObjectStorages/IObjectStorage.cpp | 8 ++-- src/Disks/ObjectStorages/IObjectStorage.h | 4 ++ .../Local/LocalObjectStorage.cpp | 10 +++-- .../ObjectStorages/Local/LocalObjectStorage.h | 2 + .../ObjectStorages/S3/S3ObjectStorage.cpp | 42 +++++++++++++++---- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 4 ++ .../ObjectStorages/Web/WebObjectStorage.cpp | 2 +- .../ObjectStorages/Web/WebObjectStorage.h | 2 + src/IO/S3/copyS3File.cpp | 10 +++-- src/IO/S3/copyS3File.h | 3 ++ .../MergeTree/DataPartStorageOnDiskBase.cpp | 9 ++-- .../MergeTree/DataPartStorageOnDiskBase.h | 4 +- src/Storages/MergeTree/IDataPartStorage.h | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 ++- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 20 +++++---- src/Storages/MergeTree/MergeTreeData.h | 5 ++- .../MergeTree/MergeTreePartsMover.cpp | 6 +-- src/Storages/MergeTree/MergeTreePartsMover.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 9 +++- src/Storages/MergeTree/localBackup.cpp | 25 +++++------ src/Storages/MergeTree/localBackup.h | 3 +- src/Storages/StorageMergeTree.cpp | 19 ++++++++- src/Storages/StorageReplicatedMergeTree.cpp | 35 ++++++++++++++-- .../02844_max_backup_bandwidth_s3.reference | 2 + .../02844_max_backup_bandwidth_s3.sh | 36 ++++++++++++++++ 51 files changed, 299 insertions(+), 112 deletions(-) create mode 100644 tests/queries/0_stateless/02844_max_backup_bandwidth_s3.reference create mode 100755 tests/queries/0_stateless/02844_max_backup_bandwidth_s3.sh diff --git a/programs/disks/CommandCopy.cpp b/programs/disks/CommandCopy.cpp index 4a7af1ced29..296fc708411 100644 --- a/programs/disks/CommandCopy.cpp +++ b/programs/disks/CommandCopy.cpp @@ -57,7 +57,7 @@ public: String relative_path_from = validatePathAndGetAsRelative(path_from); String relative_path_to = validatePathAndGetAsRelative(path_to); - disk_from->copyDirectoryContent(relative_path_from, disk_to, relative_path_to, /* settings= */ {}); + disk_from->copyDirectoryContent(relative_path_from, disk_to, relative_path_to, /* read_settings= */ {}, /* write_settings= */ {}); } }; } diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index 21b3afbddf8..1e260ad22d9 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -46,7 +46,7 @@ void BackupReaderDisk::copyFileToDisk(const String & path_in_backup, size_t file { /// Use more optimal way. LOG_TRACE(log, "Copying file {} from disk {} to disk {}", path_in_backup, disk->getName(), destination_disk->getName()); - disk->copyFile(root_path / path_in_backup, *destination_disk, destination_path, write_settings); + disk->copyFile(root_path / path_in_backup, *destination_disk, destination_path, read_settings, write_settings); return; /// copied! } } @@ -119,7 +119,7 @@ void BackupWriterDisk::copyFileFromDisk(const String & path_in_backup, DiskPtr s LOG_TRACE(log, "Copying file {} from disk {} to disk {}", src_path, src_disk->getName(), disk->getName()); auto dest_file_path = root_path / path_in_backup; disk->createDirectories(dest_file_path.parent_path()); - src_disk->copyFile(src_path, *disk, dest_file_path, write_settings); + src_disk->copyFile(src_path, *disk, dest_file_path, read_settings, write_settings); return; /// copied! } } diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index ef820784bdf..5b08683b157 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -170,6 +170,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s /* dest_bucket= */ blob_path[1], /* dest_key= */ blob_path[0], request_settings, + read_settings, object_attributes, threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupReaderS3"), /* for_disk_s3= */ true); @@ -230,6 +231,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, request_settings, + read_settings, {}, threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3")); return; /// copied! diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index ca7cbf443f2..7bc7c1c7dc4 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -324,7 +324,7 @@ ReservationPtr DiskEncrypted::reserve(UInt64 bytes) } -void DiskEncrypted::copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const WriteSettings & settings) +void DiskEncrypted::copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) { /// Check if we can copy the file without deciphering. if (isSameDiskType(*this, *to_disk)) @@ -340,14 +340,14 @@ void DiskEncrypted::copyDirectoryContent(const String & from_dir, const std::sha auto wrapped_from_path = wrappedPath(from_dir); auto to_delegate = to_disk_enc->delegate; auto wrapped_to_path = to_disk_enc->wrappedPath(to_dir); - delegate->copyDirectoryContent(wrapped_from_path, to_delegate, wrapped_to_path, settings); + delegate->copyDirectoryContent(wrapped_from_path, to_delegate, wrapped_to_path, read_settings, write_settings); return; } } } /// Copy the file through buffers with deciphering. - IDisk::copyDirectoryContent(from_dir, to_disk, to_dir, settings); + IDisk::copyDirectoryContent(from_dir, to_disk, to_dir, read_settings, write_settings); } std::unique_ptr DiskEncrypted::readFile( diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 2252e4f43f5..8b4461a8dee 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -112,7 +112,7 @@ public: delegate->listFiles(wrapped_path, file_names); } - void copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const WriteSettings & settings) override; + void copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) override; std::unique_ptr readFile( const String & path, diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index 3fd2085f9cc..daeab7aae6c 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -53,11 +53,11 @@ String DiskEncryptedSettings::findKeyByFingerprint(UInt128 key_fingerprint, cons return it->second; } -void DiskEncryptedTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path, const WriteSettings & settings) +void DiskEncryptedTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path, const ReadSettings & read_settings, const WriteSettings & write_settings) { auto wrapped_from_path = wrappedPath(from_file_path); auto wrapped_to_path = wrappedPath(to_file_path); - delegate_transaction->copyFile(wrapped_from_path, wrapped_to_path, settings); + delegate_transaction->copyFile(wrapped_from_path, wrapped_to_path, read_settings, write_settings); } std::unique_ptr DiskEncryptedTransaction::writeFile( // NOLINT diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h index 70ed1f469ef..6cb2941cc11 100644 --- a/src/Disks/DiskEncryptedTransaction.h +++ b/src/Disks/DiskEncryptedTransaction.h @@ -116,7 +116,7 @@ public: /// but it's impossible to implement correctly in transactions because other disk can /// use different metadata storage. /// TODO: maybe remove it at all, we don't want copies - void copyFile(const std::string & from_file_path, const std::string & to_file_path, const WriteSettings & settings) override; + void copyFile(const std::string & from_file_path, const std::string & to_file_path, const ReadSettings & read_settings, const WriteSettings & write_settings) override; /// Open the file for write and return WriteBufferFromFileBase object. std::unique_ptr writeFile( /// NOLINT diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index aaa22655f7b..c71f6f81de2 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -432,13 +432,13 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) return typeid(one) == typeid(another); } -void DiskLocal::copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const WriteSettings & settings) +void DiskLocal::copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) { /// If throttling was configured we cannot use copying directly. - if (isSameDiskType(*this, *to_disk) && !settings.local_throttler) + if (isSameDiskType(*this, *to_disk) && !read_settings.local_throttler && !write_settings.local_throttler) fs::copy(fs::path(disk_path) / from_dir, fs::path(to_disk->getPath()) / to_dir, fs::copy_options::recursive | fs::copy_options::overwrite_existing); /// Use more optimal way. else - IDisk::copyDirectoryContent(from_dir, to_disk, to_dir, settings); + IDisk::copyDirectoryContent(from_dir, to_disk, to_dir, read_settings, write_settings); } SyncGuardPtr DiskLocal::getDirectorySyncGuard(const String & path) const diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 197f6bb9367..c52c192d824 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -65,7 +65,7 @@ public: void replaceFile(const String & from_path, const String & to_path) override; - void copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const WriteSettings & settings) override; + void copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) override; void listFiles(const String & path, std::vector & file_names) const override; diff --git a/src/Disks/FakeDiskTransaction.h b/src/Disks/FakeDiskTransaction.h index 440ee6271e9..f83642eee56 100644 --- a/src/Disks/FakeDiskTransaction.h +++ b/src/Disks/FakeDiskTransaction.h @@ -54,9 +54,9 @@ public: disk.replaceFile(from_path, to_path); } - void copyFile(const std::string & from_file_path, const std::string & to_file_path, const WriteSettings & settings) override + void copyFile(const std::string & from_file_path, const std::string & to_file_path, const ReadSettings & read_settings, const WriteSettings & write_settings) override { - disk.copyFile(from_file_path, disk, to_file_path, settings); + disk.copyFile(from_file_path, disk, to_file_path, read_settings, write_settings); } std::unique_ptr writeFile( /// NOLINT diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 5b9f1208622..1997ce06990 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -24,13 +24,13 @@ bool IDisk::isDirectoryEmpty(const String & path) const return !iterateDirectory(path)->isValid(); } -void IDisk::copyFile(const String & from_file_path, IDisk & to_disk, const String & to_file_path, const WriteSettings & settings) /// NOLINT +void IDisk::copyFile(const String & from_file_path, IDisk & to_disk, const String & to_file_path, const ReadSettings & read_settings, const WriteSettings & write_settings) /// NOLINT { LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} (path: {}) {} to {} (path: {}) {}.", getName(), getPath(), from_file_path, to_disk.getName(), to_disk.getPath(), to_file_path); - auto in = readFile(from_file_path); - auto out = to_disk.writeFile(to_file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, settings); + auto in = readFile(from_file_path, read_settings); + auto out = to_disk.writeFile(to_file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, write_settings); copyData(*in, *out); out->finalize(); } @@ -80,7 +80,7 @@ UInt128 IDisk::getEncryptedFileIV(const String &) const using ResultsCollector = std::vector>; -void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_path, ThreadPool & pool, ResultsCollector & results, bool copy_root_dir, const WriteSettings & settings) +void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_path, ThreadPool & pool, ResultsCollector & results, bool copy_root_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) { if (from_disk.isFile(from_path)) { @@ -88,7 +88,7 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p auto future = promise->get_future(); pool.scheduleOrThrowOnError( - [&from_disk, from_path, &to_disk, to_path, &settings, promise, thread_group = CurrentThread::getGroup()]() + [&from_disk, from_path, &to_disk, to_path, &read_settings, &write_settings, promise, thread_group = CurrentThread::getGroup()]() { try { @@ -97,7 +97,7 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p if (thread_group) CurrentThread::attachToGroup(thread_group); - from_disk.copyFile(from_path, to_disk, fs::path(to_path) / fileName(from_path), settings); + from_disk.copyFile(from_path, to_disk, fs::path(to_path) / fileName(from_path), read_settings, write_settings); promise->set_value(); } catch (...) @@ -119,19 +119,19 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p } for (auto it = from_disk.iterateDirectory(from_path); it->isValid(); it->next()) - asyncCopy(from_disk, it->path(), to_disk, dest, pool, results, true, settings); + asyncCopy(from_disk, it->path(), to_disk, dest, pool, results, true, read_settings, write_settings); } } -void IDisk::copyThroughBuffers(const String & from_path, const std::shared_ptr & to_disk, const String & to_path, bool copy_root_dir, WriteSettings settings) +void IDisk::copyThroughBuffers(const String & from_path, const std::shared_ptr & to_disk, const String & to_path, bool copy_root_dir, const ReadSettings & read_settings, WriteSettings write_settings) { ResultsCollector results; /// Disable parallel write. We already copy in parallel. /// Avoid high memory usage. See test_s3_zero_copy_ttl/test.py::test_move_and_s3_memory_usage - settings.s3_allow_parallel_part_upload = false; + write_settings.s3_allow_parallel_part_upload = false; - asyncCopy(*this, from_path, *to_disk, to_path, copying_thread_pool, results, copy_root_dir, settings); + asyncCopy(*this, from_path, *to_disk, to_path, copying_thread_pool, results, copy_root_dir, read_settings, write_settings); for (auto & result : results) result.wait(); @@ -140,12 +140,12 @@ void IDisk::copyThroughBuffers(const String & from_path, const std::shared_ptr & to_disk, const String & to_dir, const WriteSettings & settings) +void IDisk::copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) { if (!to_disk->exists(to_dir)) to_disk->createDirectories(to_dir); - copyThroughBuffers(from_dir, to_disk, to_dir, /* copy_root_dir= */ false, settings); + copyThroughBuffers(from_dir, to_disk, to_dir, /* copy_root_dir= */ false, read_settings, write_settings); } void IDisk::truncateFile(const String &, size_t) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index fc4eaec428c..bfb418e1c5e 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -193,14 +193,15 @@ public: virtual void replaceFile(const String & from_path, const String & to_path) = 0; /// Recursively copy files from from_dir to to_dir. Create to_dir if not exists. - virtual void copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const WriteSettings & settings); + virtual void copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings); /// Copy file `from_file_path` to `to_file_path` located at `to_disk`. virtual void copyFile( /// NOLINT const String & from_file_path, IDisk & to_disk, const String & to_file_path, - const WriteSettings & settings = {}); + const ReadSettings & read_settings = {}, + const WriteSettings & write_settings = {}); /// List files at `path` and add their names to `file_names` virtual void listFiles(const String & path, std::vector & file_names) const = 0; @@ -470,7 +471,7 @@ protected: /// Base implementation of the function copy(). /// It just opens two files, reads data by portions from the first file, and writes it to the second one. /// A derived class may override copy() to provide a faster implementation. - void copyThroughBuffers(const String & from_path, const std::shared_ptr & to_disk, const String & to_path, bool copy_root_dir, WriteSettings settings); + void copyThroughBuffers(const String & from_path, const std::shared_ptr & to_disk, const String & to_path, bool copy_root_dir, const ReadSettings & read_settings, WriteSettings write_settings); virtual void checkAccessImpl(const String & path); diff --git a/src/Disks/IDiskTransaction.h b/src/Disks/IDiskTransaction.h index 9f18206a4ad..975c41cb70b 100644 --- a/src/Disks/IDiskTransaction.h +++ b/src/Disks/IDiskTransaction.h @@ -59,7 +59,11 @@ public: /// but it's impossible to implement correctly in transactions because other disk can /// use different metadata storage. /// TODO: maybe remove it at all, we don't want copies - virtual void copyFile(const std::string & from_file_path, const std::string & to_file_path, const WriteSettings & settings = {}) = 0; + virtual void copyFile( + const std::string & from_file_path, + const std::string & to_file_path, + const ReadSettings & read_settings = {}, + const WriteSettings & write_settings = {}) = 0; /// Open the file for write and return WriteBufferFromFileBase object. virtual std::unique_ptr writeFile( /// NOLINT diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index f76fbd45736..73be834c1bb 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -357,6 +357,8 @@ ObjectMetadata AzureObjectStorage::getObjectMetadata(const std::string & path) c void AzureObjectStorage::copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings &, + const WriteSettings &, std::optional object_to_attributes) { auto client_ptr = client.get(); diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index b5f81cef235..5436860818c 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -100,6 +100,8 @@ public: void copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, std::optional object_to_attributes = {}) override; void shutdown() override {} diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index 0da572a06ab..d94c26f27e8 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -160,16 +160,22 @@ void CachedObjectStorage::removeObjectsIfExist(const StoredObjects & objects) void CachedObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, IObjectStorage & object_storage_to, std::optional object_to_attributes) { - object_storage->copyObjectToAnotherObjectStorage(object_from, object_to, object_storage_to, object_to_attributes); + object_storage->copyObjectToAnotherObjectStorage(object_from, object_to, read_settings, write_settings, object_storage_to, object_to_attributes); } void CachedObjectStorage::copyObject( // NOLINT - const StoredObject & object_from, const StoredObject & object_to, std::optional object_to_attributes) + const StoredObject & object_from, + const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + std::optional object_to_attributes) { - object_storage->copyObject(object_from, object_to, object_to_attributes); + object_storage->copyObject(object_from, object_to, read_settings, write_settings, object_to_attributes); } std::unique_ptr CachedObjectStorage::cloneObjectStorage( diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 76f16c9d930..925abbc6932 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -57,11 +57,15 @@ public: void copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, std::optional object_to_attributes = {}) override; void copyObjectToAnotherObjectStorage( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, IObjectStorage & object_storage_to, std::optional object_to_attributes = {}) override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 466a1d3d5dd..734482ae851 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -68,7 +68,7 @@ DiskObjectStorage::DiskObjectStorage( , send_metadata(config.getBool(config_prefix + ".send_metadata", false)) , read_resource_name(config.getString(config_prefix + ".read_resource", "")) , write_resource_name(config.getString(config_prefix + ".write_resource", "")) - , metadata_helper(std::make_unique(this, ReadSettings{})) + , metadata_helper(std::make_unique(this, ReadSettings{}, WriteSettings{})) {} StoredObjects DiskObjectStorage::getStorageObjects(const String & local_path) const @@ -180,7 +180,8 @@ void DiskObjectStorage::copyFile( /// NOLINT const String & from_file_path, IDisk & to_disk, const String & to_file_path, - const WriteSettings & settings) + const ReadSettings & read_settings, + const WriteSettings & write_settings) { if (this == &to_disk) { @@ -192,7 +193,7 @@ void DiskObjectStorage::copyFile( /// NOLINT else { /// Copy through buffers - IDisk::copyFile(from_file_path, to_disk, to_file_path, settings); + IDisk::copyFile(from_file_path, to_disk, to_file_path, read_settings, write_settings); } } diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 72103edd77e..ccd7e807513 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -162,7 +162,8 @@ public: const String & from_file_path, IDisk & to_disk, const String & to_file_path, - const WriteSettings & settings = {}) override; + const ReadSettings & read_settings = {}, + const WriteSettings & write_settings = {}) override; void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context_, const String &, const DisksMap &) override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp index bbcdd40d85f..91e15547068 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp @@ -84,7 +84,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::saveSchemaVersion(const int & { StoredObject object{fs::path(disk->object_storage_root_path) / SCHEMA_VERSION_OBJECT}; - auto buf = disk->object_storage->writeObject(object, WriteMode::Rewrite); + auto buf = disk->object_storage->writeObject(object, WriteMode::Rewrite, /* attributes= */ {}, /* buf_size= */ DBMS_DEFAULT_BUFFER_SIZE, write_settings); writeIntText(version, *buf); buf->finalize(); @@ -93,7 +93,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::saveSchemaVersion(const int & void DiskObjectStorageRemoteMetadataRestoreHelper::updateObjectMetadata(const String & key, const ObjectAttributes & metadata) const { StoredObject object{key}; - disk->object_storage->copyObject(object, object, metadata); + disk->object_storage->copyObject(object, object, read_settings, write_settings, metadata); } void DiskObjectStorageRemoteMetadataRestoreHelper::migrateFileToRestorableSchema(const String & path) const @@ -434,7 +434,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::processRestoreFiles( /// Copy object if we restore to different bucket / path. if (source_object_storage->getObjectsNamespace() != disk->object_storage->getObjectsNamespace() || disk->object_storage_root_path != source_path) - source_object_storage->copyObjectToAnotherObjectStorage(object_from, object_to, *disk->object_storage); + source_object_storage->copyObjectToAnotherObjectStorage(object_from, object_to, read_settings, write_settings, *disk->object_storage); auto tx = disk->metadata_storage->createTransaction(); tx->addBlobToMetadata(path, relative_key, meta.size_bytes); diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h index e7de4afcaf3..ee81e8a209e 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h @@ -24,9 +24,10 @@ public: static constexpr UInt64 LATEST_REVISION = std::numeric_limits::max(); static constexpr UInt64 UNKNOWN_REVISION = 0; - DiskObjectStorageRemoteMetadataRestoreHelper(DiskObjectStorage * disk_, ReadSettings read_settings_) + DiskObjectStorageRemoteMetadataRestoreHelper(DiskObjectStorage * disk_, ReadSettings read_settings_, WriteSettings write_settings_) : disk(disk_) , read_settings(std::move(read_settings_)) + , write_settings(std::move(write_settings_)) , operation_log_suffix("-" + getFQDNOrHostName()) { } @@ -94,6 +95,7 @@ private: ObjectStoragePtr object_storage_from_another_namespace; ReadSettings read_settings; + WriteSettings write_settings; String operation_log_suffix; }; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index fd01caacd25..99cbd234e08 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -474,6 +475,9 @@ struct WriteFileObjectStorageOperation final : public IDiskObjectStorageOperatio struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation { + ReadSettings read_settings; + WriteSettings write_settings; + /// Local paths std::string from_path; std::string to_path; @@ -483,9 +487,13 @@ struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation CopyFileObjectStorageOperation( IObjectStorage & object_storage_, IMetadataStorage & metadata_storage_, + const ReadSettings & read_settings_, + const WriteSettings & write_settings_, const std::string & from_path_, const std::string & to_path_) : IDiskObjectStorageOperation(object_storage_, metadata_storage_) + , read_settings(read_settings_) + , write_settings(write_settings_) , from_path(from_path_) , to_path(to_path_) {} @@ -505,7 +513,7 @@ struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation std::string blob_name = object_storage.generateBlobNameForPath(to_path); auto object_to = StoredObject(fs::path(metadata_storage.getObjectStorageRootPath()) / blob_name); - object_storage.copyObject(object_from, object_to); + object_storage.copyObject(object_from, object_to, read_settings, write_settings); tx->addBlobToMetadata(to_path, blob_name, object_from.bytes_size); @@ -810,13 +818,10 @@ void DiskObjectStorageTransaction::createFile(const std::string & path) })); } -void DiskObjectStorageTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path, const WriteSettings & settings) +void DiskObjectStorageTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path, const ReadSettings & read_settings, const WriteSettings & write_settings) { - /// NOTE: For native copy we can ignore throttling, so no need to use WriteSettings - UNUSED(settings); - operations_to_execute.emplace_back( - std::make_unique(object_storage, metadata_storage, from_file_path, to_file_path)); + std::make_unique(object_storage, metadata_storage, read_settings, write_settings, from_file_path, to_file_path)); } void DiskObjectStorageTransaction::commit() diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h index 8ce10dad212..4b62a41e161 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h @@ -86,7 +86,7 @@ public: void createFile(const String & path) override; - void copyFile(const std::string & from_file_path, const std::string & to_file_path, const WriteSettings & settings) override; + void copyFile(const std::string & from_file_path, const std::string & to_file_path, const ReadSettings & read_settings, const WriteSettings &) override; /// writeFile is a difficult function for transactions. /// Now it's almost noop because metadata added to transaction in finalize method diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 60230ce2fb0..5eca98aa494 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -133,6 +133,8 @@ ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string &) const void HDFSObjectStorage::copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, std::optional object_to_attributes) { if (object_to_attributes.has_value()) @@ -140,8 +142,8 @@ void HDFSObjectStorage::copyObject( /// NOLINT ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects"); - auto in = readObject(object_from); - auto out = writeObject(object_to, WriteMode::Rewrite); + auto in = readObject(object_from, read_settings); + auto out = writeObject(object_to, WriteMode::Rewrite, /* attributes= */ {}, /* buf_size= */ DBMS_DEFAULT_BUFFER_SIZE, write_settings); copyData(*in, *out); out->finalize(); } diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index a691b089b43..8d770c12d8f 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -98,6 +98,8 @@ public: void copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, std::optional object_to_attributes = {}) override; void shutdown() override; diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index ea22294224c..3c77de8f5b7 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -62,14 +62,16 @@ ThreadPool & IObjectStorage::getThreadPoolWriter() void IObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, IObjectStorage & object_storage_to, std::optional object_to_attributes) { if (&object_storage_to == this) - copyObject(object_from, object_to, object_to_attributes); + copyObject(object_from, object_to, read_settings, write_settings, object_to_attributes); - auto in = readObject(object_from); - auto out = object_storage_to.writeObject(object_to, WriteMode::Rewrite); + auto in = readObject(object_from, read_settings); + auto out = object_storage_to.writeObject(object_to, WriteMode::Rewrite, /* attributes= */ {}, /* buf_size= */ DBMS_DEFAULT_BUFFER_SIZE, write_settings); copyData(*in, *out); out->finalize(); } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 32f9d1ba764..032795b380f 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -131,6 +131,8 @@ public: virtual void copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, std::optional object_to_attributes = {}) = 0; /// Copy object to another instance of object storage @@ -139,6 +141,8 @@ public: virtual void copyObjectToAnotherObjectStorage( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, IObjectStorage & object_storage_to, std::optional object_to_attributes = {}); diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 69ccf309096..cc53df956c6 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -167,10 +167,14 @@ ObjectMetadata LocalObjectStorage::getObjectMetadata(const std::string & /* path } void LocalObjectStorage::copyObject( // NOLINT - const StoredObject & object_from, const StoredObject & object_to, std::optional /* object_to_attributes */) + const StoredObject & object_from, + const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + std::optional /* object_to_attributes */) { - auto in = readObject(object_from); - auto out = writeObject(object_to, WriteMode::Rewrite); + auto in = readObject(object_from, read_settings); + auto out = writeObject(object_to, WriteMode::Rewrite, /* attributes= */ {}, /* buf_size= */ DBMS_DEFAULT_BUFFER_SIZE, write_settings); copyData(*in, *out); out->finalize(); } diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h index 630320ab7f9..aa3a68731e4 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h @@ -57,6 +57,8 @@ public: void copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, std::optional object_to_attributes = {}) override; void shutdown() override; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 0d9670efebe..8f020e0d1ac 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -425,6 +425,8 @@ ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) cons void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, IObjectStorage & object_storage_to, std::optional object_to_attributes) { @@ -435,24 +437,48 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(clients_->client, clients_->client_with_long_timeout, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, - settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); + copyS3File(clients_->client, + clients_->client_with_long_timeout, + bucket, + object_from.remote_path, + 0, + size, + dest_s3->bucket, + object_to.remote_path, + settings_ptr->request_settings, + patchSettings(read_settings), + object_to_attributes, + scheduler, + /* for_disk_s3= */ true); } else - { - IObjectStorage::copyObjectToAnotherObjectStorage(object_from, object_to, object_storage_to, object_to_attributes); - } + IObjectStorage::copyObjectToAnotherObjectStorage(object_from, object_to, read_settings, write_settings, object_storage_to, object_to_attributes); } void S3ObjectStorage::copyObject( // NOLINT - const StoredObject & object_from, const StoredObject & object_to, std::optional object_to_attributes) + const StoredObject & object_from, + const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings &, + std::optional object_to_attributes) { auto clients_ = clients.get(); auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(clients_->client, clients_->client_with_long_timeout, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, - settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); + copyS3File(clients_->client, + clients_->client_with_long_timeout, + bucket, + object_from.remote_path, + 0, + size, + bucket, + object_to.remote_path, + settings_ptr->request_settings, + patchSettings(read_settings), + object_to_attributes, + scheduler, + /* for_disk_s3= */ true); } void S3ObjectStorage::setNewSettings(std::unique_ptr && s3_settings_) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 527b1479d89..6e516b39c88 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -135,11 +135,15 @@ public: void copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, std::optional object_to_attributes = {}) override; void copyObjectToAnotherObjectStorage( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, IObjectStorage & object_storage_to, std::optional object_to_attributes = {}) override; diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 45b183c15f2..ea05012fb61 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -264,7 +264,7 @@ void WebObjectStorage::removeObjectsIfExist(const StoredObjects &) throwNotAllowed(); } -void WebObjectStorage::copyObject(const StoredObject &, const StoredObject &, std::optional) // NOLINT +void WebObjectStorage::copyObject(const StoredObject &, const StoredObject &, const ReadSettings &, const WriteSettings &, std::optional) // NOLINT { throwNotAllowed(); } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h index 1a21d94e230..089bdb99e71 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h @@ -68,6 +68,8 @@ public: void copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, std::optional object_to_attributes = {}) override; void shutdown() override; diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 002b8dde566..a16a1a41505 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -610,6 +610,7 @@ namespace const String & dest_bucket_, const String & dest_key_, const S3Settings::RequestSettings & request_settings_, + const ReadSettings & read_settings_, const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, bool for_disk_s3_) @@ -619,6 +620,7 @@ namespace , offset(src_offset_) , size(src_size_) , supports_multipart_copy(client_ptr_->supportsMultiPartCopy()) + , read_settings(read_settings_) { } @@ -639,12 +641,13 @@ namespace size_t offset; size_t size; bool supports_multipart_copy; + const ReadSettings read_settings; CreateReadBuffer getSourceObjectReadBuffer() { return [&] { - return std::make_unique(client_ptr, src_bucket, src_key, "", request_settings, Context::getGlobalContextInstance()->getReadSettings()); + return std::make_unique(client_ptr, src_bucket, src_key, "", request_settings, read_settings); }; } @@ -826,20 +829,21 @@ void copyS3File( const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, + const ReadSettings & read_settings, const std::optional> & object_metadata, ThreadPoolCallbackRunner schedule, bool for_disk_s3) { if (settings.allow_native_copy) { - CopyFileHelper helper{s3_client, s3_client_with_long_timeout, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; + CopyFileHelper helper{s3_client, s3_client_with_long_timeout, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, read_settings, object_metadata, schedule, for_disk_s3}; helper.performCopy(); } else { auto create_read_buffer = [&] { - return std::make_unique(s3_client, src_bucket, src_key, "", settings, Context::getGlobalContextInstance()->getReadSettings()); + return std::make_unique(s3_client, src_bucket, src_key, "", settings, read_settings); }; copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, s3_client_with_long_timeout, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); } diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 3477f5a20ab..1bcbfd7735e 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -31,6 +31,8 @@ using CreateReadBuffer = std::function()>; /// CompleteMultipartUpload requests. These requests need longer timeout because S3 servers often /// block on them for multiple seconds without sending or receiving data from us (maybe the servers /// are copying data internally, or maybe throttling, idk). +/// +/// read_settings - is used for throttling in case of native copy is not possible void copyS3File( const std::shared_ptr & s3_client, const std::shared_ptr & s3_client_with_long_timeout, @@ -41,6 +43,7 @@ void copyS3File( const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, + const ReadSettings & read_settings, const std::optional> & object_metadata = std::nullopt, ThreadPoolCallbackRunner schedule_ = {}, bool for_disk_s3 = false); diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index 27d8991bd62..7fc8187aee5 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -416,7 +416,8 @@ void DataPartStorageOnDiskBase::backup( MutableDataPartStoragePtr DataPartStorageOnDiskBase::freeze( const std::string & to, const std::string & dir_path, - const WriteSettings & settings, + const ReadSettings & read_settings, + const WriteSettings & write_settings, std::function save_metadata_callback, const ClonePartParams & params) const { @@ -430,7 +431,8 @@ MutableDataPartStoragePtr DataPartStorageOnDiskBase::freeze( disk, getRelativePath(), fs::path(to) / dir_path, - settings, + read_settings, + write_settings, params.make_source_readonly, /* max_level= */ {}, params.copy_instead_of_hardlink, @@ -466,6 +468,7 @@ MutableDataPartStoragePtr DataPartStorageOnDiskBase::clonePart( const std::string & to, const std::string & dir_path, const DiskPtr & dst_disk, + const ReadSettings & read_settings, const WriteSettings & write_settings, Poco::Logger * log) const { @@ -482,7 +485,7 @@ MutableDataPartStoragePtr DataPartStorageOnDiskBase::clonePart( try { dst_disk->createDirectories(to); - src_disk->copyDirectoryContent(getRelativePath(), dst_disk, path_to_clone, write_settings); + src_disk->copyDirectoryContent(getRelativePath(), dst_disk, path_to_clone, read_settings, write_settings); } catch (...) { diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 0adf048b56a..1826e84c28d 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -63,7 +63,8 @@ public: MutableDataPartStoragePtr freeze( const std::string & to, const std::string & dir_path, - const WriteSettings & settings, + const ReadSettings & read_settings, + const WriteSettings & write_settings, std::function save_metadata_callback, const ClonePartParams & params) const override; @@ -71,6 +72,7 @@ public: const std::string & to, const std::string & dir_path, const DiskPtr & dst_disk, + const ReadSettings & read_settings, const WriteSettings & write_settings, Poco::Logger * log) const override; diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index c76b17f3370..072cb29626e 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -250,7 +250,8 @@ public: virtual std::shared_ptr freeze( const std::string & to, const std::string & dir_path, - const WriteSettings & settings, + const ReadSettings & read_settings, + const WriteSettings & write_settings, std::function save_metadata_callback, const ClonePartParams & params) const = 0; @@ -259,6 +260,7 @@ public: const std::string & to, const std::string & dir_path, const DiskPtr & disk, + const ReadSettings & read_settings, const WriteSettings & write_settings, Poco::Logger * log) const = 0; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 54a169fc779..dc387496371 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1793,12 +1793,13 @@ DataPartStoragePtr IMergeTreeDataPart::makeCloneInDetached(const String & prefix return getDataPartStorage().freeze( storage.relative_data_path, *maybe_path_in_detached, + Context::getGlobalContextInstance()->getReadSettings(), Context::getGlobalContextInstance()->getWriteSettings(), /* save_metadata_callback= */ {}, params); } -MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & directory_name, const WriteSettings & write_settings) const +MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & directory_name, const ReadSettings & read_settings, const WriteSettings & write_settings) const { assertOnDisk(); @@ -1808,7 +1809,7 @@ MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & di throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not clone data part {} to empty directory.", name); String path_to_clone = fs::path(storage.relative_data_path) / directory_name / ""; - return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, write_settings, storage.log); + return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, storage.log); } UInt64 IMergeTreeDataPart::getIndexSizeFromFile() const diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 02c838458f9..c30accbc1ba 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -377,7 +377,7 @@ public: const DiskTransactionPtr & disk_transaction) const; /// Makes full clone of part in specified subdirectory (relative to storage data directory, e.g. "detached") on another disk - MutableDataPartStoragePtr makeCloneOnDisk(const DiskPtr & disk, const String & directory_name, const WriteSettings & write_settings) const; + MutableDataPartStoragePtr makeCloneOnDisk(const DiskPtr & disk, const String & directory_name, const ReadSettings & read_settings, const WriteSettings & write_settings) const; /// Checks that .bin and .mrk files exist. /// diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 14c9961f6c3..26d110f5510 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4968,7 +4968,7 @@ void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & throw Exception(ErrorCodes::UNKNOWN_DISK, "All parts of partition '{}' are already on disk '{}'", partition_id, disk->getName()); } - MovePartsOutcome moves_outcome = movePartsToSpace(parts, std::static_pointer_cast(disk), local_context->getWriteSettings()); + MovePartsOutcome moves_outcome = movePartsToSpace(parts, std::static_pointer_cast(disk), local_context->getReadSettings(), local_context->getWriteSettings()); switch (moves_outcome) { case MovePartsOutcome::MovesAreCancelled: @@ -5031,7 +5031,7 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String throw Exception(ErrorCodes::UNKNOWN_DISK, "All parts of partition '{}' are already on volume '{}'", partition_id, volume->getName()); } - MovePartsOutcome moves_outcome = movePartsToSpace(parts, std::static_pointer_cast(volume), local_context->getWriteSettings()); + MovePartsOutcome moves_outcome = movePartsToSpace(parts, std::static_pointer_cast(volume), local_context->getReadSettings(), local_context->getWriteSettings()); switch (moves_outcome) { case MovePartsOutcome::MovesAreCancelled: @@ -7488,6 +7488,7 @@ std::pair MergeTreeData::cloneAn const MergeTreePartInfo & dst_part_info, const StorageMetadataPtr & metadata_snapshot, const IDataPartStorage::ClonePartParams & params, + const ReadSettings & read_settings, const WriteSettings & write_settings) { /// Check that the storage policy contains the disk where the src_part is located. @@ -7545,6 +7546,7 @@ std::pair MergeTreeData::cloneAn auto dst_part_storage = src_part_storage->freeze( relative_data_path, tmp_dst_part_name, + read_settings, write_settings, /* save_metadata_callback= */ {}, params); @@ -7803,6 +7805,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( auto new_storage = data_part_storage->freeze( backup_part_path, part->getDataPartStorage().getPartDirectory(), + local_context->getReadSettings(), local_context->getWriteSettings(), callback, params); @@ -8002,8 +8005,9 @@ bool MergeTreeData::scheduleDataMovingJob(BackgroundJobsAssignee & assignee) assignee.scheduleMoveTask(std::make_shared( [this, moving_tagger] () mutable { + ReadSettings read_settings = Context::getGlobalContextInstance()->getReadSettings(); WriteSettings write_settings = Context::getGlobalContextInstance()->getWriteSettings(); - return moveParts(moving_tagger, write_settings, /* wait_for_move_if_zero_copy= */ false) == MovePartsOutcome::PartsMoved; + return moveParts(moving_tagger, read_settings, write_settings, /* wait_for_move_if_zero_copy= */ false) == MovePartsOutcome::PartsMoved; }, moves_assignee_trigger, getStorageID())); return true; } @@ -8018,7 +8022,7 @@ bool MergeTreeData::areBackgroundMovesNeeded() const return policy->getVolumes().size() == 1 && policy->getVolumes()[0]->getDisks().size() > 1; } -MovePartsOutcome MergeTreeData::movePartsToSpace(const DataPartsVector & parts, SpacePtr space, const WriteSettings & write_settings) +MovePartsOutcome MergeTreeData::movePartsToSpace(const DataPartsVector & parts, SpacePtr space, const ReadSettings & read_settings, const WriteSettings & write_settings) { if (parts_mover.moves_blocker.isCancelled()) return MovePartsOutcome::MovesAreCancelled; @@ -8027,7 +8031,7 @@ MovePartsOutcome MergeTreeData::movePartsToSpace(const DataPartsVector & parts, if (moving_tagger->parts_to_move.empty()) return MovePartsOutcome::NothingToMove; - return moveParts(moving_tagger, write_settings, /* wait_for_move_if_zero_copy= */ true); + return moveParts(moving_tagger, read_settings, write_settings, /* wait_for_move_if_zero_copy= */ true); } MergeTreeData::CurrentlyMovingPartsTaggerPtr MergeTreeData::selectPartsForMove() @@ -8082,7 +8086,7 @@ MergeTreeData::CurrentlyMovingPartsTaggerPtr MergeTreeData::checkPartsForMove(co return std::make_shared(std::move(parts_to_move), *this); } -MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const WriteSettings & write_settings, bool wait_for_move_if_zero_copy) +MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const ReadSettings & read_settings, const WriteSettings & write_settings, bool wait_for_move_if_zero_copy) { LOG_INFO(log, "Got {} parts to move.", moving_tagger->parts_to_move.size()); @@ -8143,7 +8147,7 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & { if (lock->isLocked()) { - cloned_part = parts_mover.clonePart(moving_part, write_settings); + cloned_part = parts_mover.clonePart(moving_part, read_settings, write_settings); parts_mover.swapClonedPart(cloned_part); break; } @@ -8170,7 +8174,7 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & } else /// Ordinary move as it should be { - cloned_part = parts_mover.clonePart(moving_part, write_settings); + cloned_part = parts_mover.clonePart(moving_part, read_settings, write_settings); parts_mover.swapClonedPart(cloned_part); } write_part_log({}); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 6f9779bde00..414fa493085 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -848,6 +848,7 @@ public: const MergeTreePartInfo & dst_part_info, const StorageMetadataPtr & metadata_snapshot, const IDataPartStorage::ClonePartParams & params, + const ReadSettings & read_settings, const WriteSettings & write_settings); virtual std::vector getMutationsStatus() const = 0; @@ -1340,7 +1341,7 @@ protected: /// MergeTree because they store mutations in different way. virtual std::map getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; /// Moves part to specified space, used in ALTER ... MOVE ... queries - MovePartsOutcome movePartsToSpace(const DataPartsVector & parts, SpacePtr space, const WriteSettings & write_settings); + MovePartsOutcome movePartsToSpace(const DataPartsVector & parts, SpacePtr space, const ReadSettings & read_settings, const WriteSettings & write_settings); struct PartBackupEntries { @@ -1494,7 +1495,7 @@ private: using CurrentlyMovingPartsTaggerPtr = std::shared_ptr; /// Move selected parts to corresponding disks - MovePartsOutcome moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const WriteSettings & write_settings, bool wait_for_move_if_zero_copy); + MovePartsOutcome moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const ReadSettings & read_settings, const WriteSettings & write_settings, bool wait_for_move_if_zero_copy); /// Select parts for move and disks for them. Used in background moving processes. CurrentlyMovingPartsTaggerPtr selectPartsForMove(); diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 51e4cee19f8..f4dc6c8d042 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -208,7 +208,7 @@ bool MergeTreePartsMover::selectPartsForMove( return false; } -MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const MergeTreeMoveEntry & moving_part, const WriteSettings & write_settings) const +MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const MergeTreeMoveEntry & moving_part, const ReadSettings & read_settings, const WriteSettings & write_settings) const { if (moves_blocker.isCancelled()) throw Exception(ErrorCodes::ABORTED, "Cancelled moving parts."); @@ -249,12 +249,12 @@ MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const Me { LOG_INFO(log, "Part {} was not fetched, we are the first who move it to another disk, so we will copy it", part->name); cloned_part_storage = part->getDataPartStorage().clonePart( - path_to_clone, part->getDataPartStorage().getPartDirectory(), disk, write_settings, log); + path_to_clone, part->getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, log); } } else { - cloned_part_storage = part->makeCloneOnDisk(disk, MergeTreeData::MOVING_DIR_NAME, write_settings); + cloned_part_storage = part->makeCloneOnDisk(disk, MergeTreeData::MOVING_DIR_NAME, read_settings, write_settings); } MergeTreeDataPartBuilder builder(*data, part->name, cloned_part_storage); diff --git a/src/Storages/MergeTree/MergeTreePartsMover.h b/src/Storages/MergeTree/MergeTreePartsMover.h index 5dcc364a4e9..f172dade40e 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.h +++ b/src/Storages/MergeTree/MergeTreePartsMover.h @@ -65,7 +65,7 @@ public: const std::lock_guard & moving_parts_lock); /// Copies part to selected reservation in detached folder. Throws exception if part already exists. - TemporaryClonedPart clonePart(const MergeTreeMoveEntry & moving_part, const WriteSettings & write_settings) const; + TemporaryClonedPart clonePart(const MergeTreeMoveEntry & moving_part, const ReadSettings & read_settings, const WriteSettings & write_settings) const; /// Replaces cloned part from detached directory into active data parts set. /// Replacing part changes state to DeleteOnDestroy and will be removed from disk after destructor of diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 808ece8dc82..15ca2b65731 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1845,7 +1845,14 @@ bool MutateTask::prepare() .txn = ctx->txn, .hardlinked_files = &ctx->hardlinked_files, .files_to_copy_instead_of_hardlinks = std::move(files_to_copy_instead_of_hardlinks), .keep_metadata_version = true }; - auto [part, lock] = ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, prefix, ctx->future_part->part_info, ctx->metadata_snapshot, clone_params, ctx->context->getWriteSettings()); + auto [part, lock] = ctx->data->cloneAndLoadDataPartOnSameDisk( + ctx->source_part, + prefix, + ctx->future_part->part_info, + ctx->metadata_snapshot, + clone_params, + ctx->context->getReadSettings(), + ctx->context->getWriteSettings()); part->getDataPartStorage().beginTransaction(); ctx->temporary_directory_lock = std::move(lock); diff --git a/src/Storages/MergeTree/localBackup.cpp b/src/Storages/MergeTree/localBackup.cpp index 4c645a8628e..c84e13b167f 100644 --- a/src/Storages/MergeTree/localBackup.cpp +++ b/src/Storages/MergeTree/localBackup.cpp @@ -21,7 +21,8 @@ void localBackupImpl( IDiskTransaction * transaction, const String & source_path, const String & destination_path, - const WriteSettings & settings, + const ReadSettings & read_settings, + const WriteSettings & write_settings, bool make_source_readonly, size_t level, std::optional max_level, @@ -56,13 +57,9 @@ void localBackupImpl( if (copy_instead_of_hardlinks || files_to_copy_instead_of_hardlinks.contains(it->name())) { if (transaction) - { - transaction->copyFile(source, destination, settings); - } + transaction->copyFile(source, destination, read_settings, write_settings); else - { - disk->copyFile(source, *disk, destination, settings); - } + disk->copyFile(source, *disk, destination, read_settings, write_settings); } else { @@ -79,7 +76,8 @@ void localBackupImpl( transaction, source, destination, - settings, + read_settings, + write_settings, make_source_readonly, level + 1, max_level, @@ -129,7 +127,8 @@ void localBackup( const DiskPtr & disk, const String & source_path, const String & destination_path, - const WriteSettings & settings, + const ReadSettings & read_settings, + const WriteSettings & write_settings, bool make_source_readonly, std::optional max_level, bool copy_instead_of_hardlinks, @@ -160,7 +159,8 @@ void localBackup( disk_transaction.get(), source_path, destination_path, - settings, + read_settings, + write_settings, make_source_readonly, /* level= */ 0, max_level, @@ -170,7 +170,7 @@ void localBackup( else if (copy_instead_of_hardlinks) { CleanupOnFail cleanup([disk, destination_path]() { disk->removeRecursive(destination_path); }); - disk->copyDirectoryContent(source_path, disk, destination_path, settings); + disk->copyDirectoryContent(source_path, disk, destination_path, read_settings, write_settings); cleanup.success(); } else @@ -189,7 +189,8 @@ void localBackup( disk_transaction.get(), source_path, destination_path, - settings, + read_settings, + write_settings, make_source_readonly, /* level= */ 0, max_level, diff --git a/src/Storages/MergeTree/localBackup.h b/src/Storages/MergeTree/localBackup.h index d9b7f3e8b0c..3490db9726e 100644 --- a/src/Storages/MergeTree/localBackup.h +++ b/src/Storages/MergeTree/localBackup.h @@ -28,7 +28,8 @@ struct WriteSettings; const DiskPtr & disk, const String & source_path, const String & destination_path, - const WriteSettings & settings, + const ReadSettings & read_settings, + const WriteSettings & write_settings, bool make_source_readonly = true, std::optional max_level = {}, bool copy_instead_of_hardlinks = false, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 22700712829..694ad9a49f8 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2043,7 +2043,14 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, my_metadata_snapshot, clone_params, local_context->getWriteSettings()); + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + my_metadata_snapshot, + clone_params, + local_context->getReadSettings(), + local_context->getWriteSettings()); dst_parts.emplace_back(std::move(dst_part)); dst_parts_locks.emplace_back(std::move(part_lock)); } @@ -2142,7 +2149,15 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; - auto [dst_part, part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, clone_params, local_context->getWriteSettings()); + auto [dst_part, part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + dest_metadata_snapshot, + clone_params, + local_context->getReadSettings(), + local_context->getWriteSettings() + ); dst_parts.emplace_back(std::move(dst_part)); dst_parts_locks.emplace_back(std::move(part_lock)); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7c7e6dbd42c..276db7639b9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2473,7 +2473,13 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( - part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, metadata_snapshot, clone_params, getContext()->getWriteSettings()); + part_desc->src_table_part, + TMP_PREFIX + "clone_", + part_desc->new_part_info, + metadata_snapshot, + clone_params, + getContext()->getReadSettings(), + getContext()->getWriteSettings()); part_desc->res_part = std::move(res_part); part_desc->temporary_part_lock = std::move(temporary_part_lock); } @@ -4568,7 +4574,14 @@ bool StorageReplicatedMergeTree::fetchPart( { chassert(!is_zero_copy_part(part_to_clone)); IDataPartStorage::ClonePartParams clone_params{ .keep_metadata_version = true }; - auto [cloned_part, lock] = cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info, metadata_snapshot, clone_params, getContext()->getWriteSettings()); + auto [cloned_part, lock] = cloneAndLoadDataPartOnSameDisk( + part_to_clone, + "tmp_clone_", + part_info, + metadata_snapshot, + clone_params, + getContext()->getReadSettings(), + getContext()->getWriteSettings()); part_directory_lock = std::move(lock); return cloned_part; }; @@ -7656,7 +7669,14 @@ void StorageReplicatedMergeTree::replacePartitionFrom( .copy_instead_of_hardlink = zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport(), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, metadata_snapshot, clone_params, query_context->getWriteSettings()); + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + metadata_snapshot, + clone_params, + query_context->getReadSettings(), + query_context->getWriteSettings()); src_parts.emplace_back(src_part); dst_parts.emplace_back(dst_part); dst_parts_locks.emplace_back(std::move(part_lock)); @@ -7896,7 +7916,14 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta .copy_instead_of_hardlink = zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport(), .metadata_version_to_write = dest_metadata_snapshot->getMetadataVersion() }; - auto [dst_part, dst_part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, clone_params, query_context->getWriteSettings()); + auto [dst_part, dst_part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + dest_metadata_snapshot, + clone_params, + query_context->getReadSettings(), + query_context->getWriteSettings()); src_parts.emplace_back(src_part); dst_parts.emplace_back(dst_part); diff --git a/tests/queries/0_stateless/02844_max_backup_bandwidth_s3.reference b/tests/queries/0_stateless/02844_max_backup_bandwidth_s3.reference new file mode 100644 index 00000000000..939eb45ce1b --- /dev/null +++ b/tests/queries/0_stateless/02844_max_backup_bandwidth_s3.reference @@ -0,0 +1,2 @@ +native_copy 0 +no_native_copy 1 diff --git a/tests/queries/0_stateless/02844_max_backup_bandwidth_s3.sh b/tests/queries/0_stateless/02844_max_backup_bandwidth_s3.sh new file mode 100755 index 00000000000..4650415c202 --- /dev/null +++ b/tests/queries/0_stateless/02844_max_backup_bandwidth_s3.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag: no-fasttest - requires S3 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -nm -q " + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, disk='s3_disk'; + -- reading 1e6*8 bytes with 1M bandwith it should take (8-1)/1=7 seconds + insert into data select * from numbers(1e6); +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --query_id "$query_id" -q "backup table data to S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data/backup2') SETTINGS allow_s3_native_copy=1" --max_backup_bandwidth=1M > /dev/null +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT + 'native_copy', + query_duration_ms >= 7e3 + FROM system.query_log + WHERE current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' AND type != 'QueryStart' +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --query_id "$query_id" -q "backup table data to S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data/backup3') SETTINGS allow_s3_native_copy=0" --max_backup_bandwidth=1M > /dev/null +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT + 'no_native_copy', + query_duration_ms >= 7e3 + FROM system.query_log + WHERE current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' AND type != 'QueryStart' +" From dd7096bc59c1bae48581e889f08f7ce1e8b121e4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 20 Sep 2023 18:02:59 +0200 Subject: [PATCH 57/69] Speed up fast tests a little bit --- docker/test/fasttest/run.sh | 24 +----------------------- tests/ci/fast_test_check.py | 6 +++--- 2 files changed, 4 insertions(+), 26 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 296a132d3e3..81b3289d8c4 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -271,34 +271,12 @@ case "$stage" in ;& "clone_root") clone_root - - # Pass control to the script from cloned sources, unless asked otherwise. - if ! [ -v FASTTEST_LOCAL_SCRIPT ] - then - # 'run' stage is deprecated, used for compatibility with old scripts. - # Replace with 'clone_submodules' after Nov 1, 2020. - # cd and CLICKHOUSE_DIR are also a setup for old scripts, remove as well. - # In modern script we undo it by changing back into workspace dir right - # away, see below. Remove that as well. - cd "$FASTTEST_SOURCE" - CLICKHOUSE_DIR=$(pwd) - export CLICKHOUSE_DIR - stage=run "$FASTTEST_SOURCE/docker/test/fasttest/run.sh" - exit $? - fi - ;& -"run") - # A deprecated stage that is called by old script and equivalent to everything - # after cloning root, starting with cloning submodules. ;& "clone_submodules") - # Recover after being called from the old script that changes into source directory. - # See the compatibility hacks in `clone_root` stage above. Remove at the same time, - # after Nov 1, 2020. - cd "$FASTTEST_WORKSPACE" clone_submodules 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/submodule_log.txt" ;& "run_cmake") + cd "$FASTTEST_WORKSPACE" run_cmake ;& "build") diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index f1bbf356fbb..43da7d98ef8 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -24,7 +24,7 @@ from commit_status_helper import ( format_description, ) from docker_pull_helper import get_image_with_version -from env_helper import S3_BUILDS_BUCKET, TEMP_PATH +from env_helper import S3_BUILDS_BUCKET, TEMP_PATH, REPO_COPY from get_robot_token import get_best_robot_token from pr_info import FORCE_TESTS_LABEL, PRInfo from report import TestResult, TestResults, read_test_results @@ -50,6 +50,7 @@ def get_fasttest_cmd(workspace, output_path, repo_path, pr_number, commit_sha, i f"-e PULL_REQUEST_NUMBER={pr_number} -e COMMIT_SHA={commit_sha} " f"-e COPY_CLICKHOUSE_BINARY_TO_OUTPUT=1 " f"-e SCCACHE_BUCKET={S3_BUILDS_BUCKET} -e SCCACHE_S3_KEY_PREFIX=ccache/sccache " + "-e stage=clone_submodules " f"--volume={workspace}:/fasttest-workspace --volume={repo_path}:/ClickHouse " f"--volume={output_path}:/test_output {image}" ) @@ -128,8 +129,7 @@ def main(): output_path = temp_path / "fasttest-output" output_path.mkdir(parents=True, exist_ok=True) - repo_path = temp_path / "fasttest-repo" - repo_path.mkdir(parents=True, exist_ok=True) + repo_path = Path(REPO_COPY) run_cmd = get_fasttest_cmd( workspace, From 16fc2739c665b985d24e6dff2508f6f2514d47e6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 20 Sep 2023 18:10:50 +0200 Subject: [PATCH 58/69] Use xargs to parallel submodules --- docker/test/fasttest/run.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 81b3289d8c4..44cb6fb5428 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -152,7 +152,11 @@ function clone_submodules ) git submodule sync - git submodule update --jobs=16 --depth 1 --single-branch --init "${SUBMODULES_TO_UPDATE[@]}" + git submodule init + # --jobs does not work as fast as real parallel running + printf '%s\0' "${SUBMODULES_TO_UPDATE[@]}" | \ + xargs --max-procs=100 --null --no-run-if-empty --max-args=1 \ + git submodule update --depth 1 --single-branch git submodule foreach git reset --hard git submodule foreach git checkout @ -f git submodule foreach git clean -xfd From 71c7e3c81e6532eecf44c25ae40d7e5a363d9bf3 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 20 Sep 2023 13:33:25 -0400 Subject: [PATCH 59/69] Add logging, fix thread name length --- src/Storages/StorageS3.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 288f5423c00..ad8af42d47e 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -484,7 +484,7 @@ StorageS3Source::ReadTaskIterator::ReadTaskIterator( : callback(callback_) { ThreadPool pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, max_threads_count); - auto pool_scheduler = threadPoolCallbackRunner(pool, "ReadTaskIteratorPrefetch"); + auto pool_scheduler = threadPoolCallbackRunner(pool, "S3ReadTaskItr"); std::vector> keys; for (size_t i = 0; i < max_threads_count; ++i) @@ -1070,6 +1070,7 @@ Pipe StorageS3::read( size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); num_streams = std::min(num_streams, estimated_keys_count); + LOG_INFO(&Poco::Logger::get("StorageS3"), "adjusting num_streams={}", num_streams); auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) From 20105958a87ff1080ccdc9fdffa93ca79d557974 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 20 Sep 2023 13:37:06 -0400 Subject: [PATCH 60/69] add reserve --- src/Storages/StorageS3.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ad8af42d47e..436f7a3c84a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -487,6 +487,7 @@ StorageS3Source::ReadTaskIterator::ReadTaskIterator( auto pool_scheduler = threadPoolCallbackRunner(pool, "S3ReadTaskItr"); std::vector> keys; + keys.reserve(max_threads_count); for (size_t i = 0; i < max_threads_count; ++i) keys.push_back(pool_scheduler([this] { return callback(); }, Priority{})); From 6acdd65c8e31f6b1fbaa9bd9cb31cd0a2d08e05c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 20 Sep 2023 19:07:42 +0200 Subject: [PATCH 61/69] Launch fast tests as the current user --- docker/test/fasttest/Dockerfile | 6 +++++- docker/test/fasttest/run.sh | 6 ++++++ tests/ci/fast_test_check.py | 2 +- 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index fd7a5640964..a38f59dacac 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -31,7 +31,11 @@ RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && cp /tmp/clickhouse-odbc-tmp/lib64/*.so /usr/local/lib/ \ && odbcinst -i -d -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbcinst.ini.sample \ && odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \ - && rm -rf /tmp/clickhouse-odbc-tmp + && rm -rf /tmp/clickhouse-odbc-tmp \ + && mkdir -p /var/lib/clickhouse \ + && chmod 777 /var/lib/clickhouse + +# chmod 777 to make the container user independent ENV TZ=Europe/Amsterdam RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 44cb6fb5428..5afba0b9ab1 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -28,6 +28,12 @@ FASTTEST_BUILD=$(readlink -f "${FASTTEST_BUILD:-${BUILD:-$FASTTEST_WORKSPACE/bui FASTTEST_DATA=$(readlink -f "${FASTTEST_DATA:-$FASTTEST_WORKSPACE/db-fasttest}") FASTTEST_OUTPUT=$(readlink -f "${FASTTEST_OUTPUT:-$FASTTEST_WORKSPACE}") PATH="$FASTTEST_BUILD/programs:$FASTTEST_SOURCE/tests:$PATH" +# Work around for non-existent user +if [ "$HOME" == "/" ]; then + HOME="$FASTTEST_WORKSPACE/user-home" + mkdir -p "$HOME" + export HOME +fi # Export these variables, so that all subsequent invocations of the script # use them, and not try to guess them anew, which leads to weird effects. diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 43da7d98ef8..281bf04a171 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -42,7 +42,7 @@ csv.field_size_limit(sys.maxsize) def get_fasttest_cmd(workspace, output_path, repo_path, pr_number, commit_sha, image): return ( - f"docker run --cap-add=SYS_PTRACE " + f"docker run --cap-add=SYS_PTRACE --user={os.geteuid()}:{os.getegid()} " "--network=host " # required to get access to IAM credentials f"-e FASTTEST_WORKSPACE=/fasttest-workspace -e FASTTEST_OUTPUT=/test_output " f"-e FASTTEST_SOURCE=/ClickHouse --cap-add=SYS_PTRACE " From 05a1c96258394b591e94601221bc0180305a27d6 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Wed, 20 Sep 2023 23:00:25 +0200 Subject: [PATCH 62/69] Interval operator support plural literals --- src/Parsers/parseIntervalKind.cpp | 33 ++++++++++++------- ..._operator_support_plural_literal.reference | 16 +++++++++ ...terval_operator_support_plural_literal.sql | 16 +++++++++ 3 files changed, 54 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02884_interval_operator_support_plural_literal.reference create mode 100644 tests/queries/0_stateless/02884_interval_operator_support_plural_literal.sql diff --git a/src/Parsers/parseIntervalKind.cpp b/src/Parsers/parseIntervalKind.cpp index 77c3178ae2b..fe052287083 100644 --- a/src/Parsers/parseIntervalKind.cpp +++ b/src/Parsers/parseIntervalKind.cpp @@ -7,77 +7,88 @@ namespace DB { bool parseIntervalKind(IParser::Pos & pos, Expected & expected, IntervalKind & result) { - if (ParserKeyword("NANOSECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_NANOSECOND").ignore(pos, expected) + if (ParserKeyword("NANOSECOND").ignore(pos, expected) || ParserKeyword("NANOSECONDS").ignore(pos, expected) + || ParserKeyword("SQL_TSI_NANOSECOND").ignore(pos, expected) || ParserKeyword("NS").ignore(pos, expected)) { result = IntervalKind::Nanosecond; return true; } - if (ParserKeyword("MICROSECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_MICROSECOND").ignore(pos, expected) + if (ParserKeyword("MICROSECOND").ignore(pos, expected) || ParserKeyword("MICROSECONDS").ignore(pos, expected) + || ParserKeyword("SQL_TSI_MICROSECOND").ignore(pos, expected) || ParserKeyword("MCS").ignore(pos, expected)) { result = IntervalKind::Microsecond; return true; } - if (ParserKeyword("MILLISECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_MILLISECOND").ignore(pos, expected) + if (ParserKeyword("MILLISECOND").ignore(pos, expected) || ParserKeyword("MILLISECONDS").ignore(pos, expected) + || ParserKeyword("SQL_TSI_MILLISECOND").ignore(pos, expected) || ParserKeyword("MS").ignore(pos, expected)) { result = IntervalKind::Millisecond; return true; } - if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_SECOND").ignore(pos, expected) + if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SECONDS").ignore(pos, expected) + || ParserKeyword("SQL_TSI_SECOND").ignore(pos, expected) || ParserKeyword("SS").ignore(pos, expected) || ParserKeyword("S").ignore(pos, expected)) { result = IntervalKind::Second; return true; } - if (ParserKeyword("MINUTE").ignore(pos, expected) || ParserKeyword("SQL_TSI_MINUTE").ignore(pos, expected) + if (ParserKeyword("MINUTE").ignore(pos, expected) || ParserKeyword("MINUTES").ignore(pos, expected) + || ParserKeyword("SQL_TSI_MINUTE").ignore(pos, expected) || ParserKeyword("MI").ignore(pos, expected) || ParserKeyword("N").ignore(pos, expected)) { result = IntervalKind::Minute; return true; } - if (ParserKeyword("HOUR").ignore(pos, expected) || ParserKeyword("SQL_TSI_HOUR").ignore(pos, expected) + if (ParserKeyword("HOUR").ignore(pos, expected) || ParserKeyword("HOURS").ignore(pos, expected) + || ParserKeyword("SQL_TSI_HOUR").ignore(pos, expected) || ParserKeyword("HH").ignore(pos, expected) || ParserKeyword("H").ignore(pos, expected)) { result = IntervalKind::Hour; return true; } - if (ParserKeyword("DAY").ignore(pos, expected) || ParserKeyword("SQL_TSI_DAY").ignore(pos, expected) + if (ParserKeyword("DAY").ignore(pos, expected) || ParserKeyword("DAYS").ignore(pos, expected) + || ParserKeyword("SQL_TSI_DAY").ignore(pos, expected) || ParserKeyword("DD").ignore(pos, expected) || ParserKeyword("D").ignore(pos, expected)) { result = IntervalKind::Day; return true; } - if (ParserKeyword("WEEK").ignore(pos, expected) || ParserKeyword("SQL_TSI_WEEK").ignore(pos, expected) + if (ParserKeyword("WEEK").ignore(pos, expected) || ParserKeyword("WEEKS").ignore(pos, expected) + || ParserKeyword("SQL_TSI_WEEK").ignore(pos, expected) || ParserKeyword("WK").ignore(pos, expected) || ParserKeyword("WW").ignore(pos, expected)) { result = IntervalKind::Week; return true; } - if (ParserKeyword("MONTH").ignore(pos, expected) || ParserKeyword("SQL_TSI_MONTH").ignore(pos, expected) + if (ParserKeyword("MONTH").ignore(pos, expected) || ParserKeyword("MONTHS").ignore(pos, expected) + || ParserKeyword("SQL_TSI_MONTH").ignore(pos, expected) || ParserKeyword("MM").ignore(pos, expected) || ParserKeyword("M").ignore(pos, expected)) { result = IntervalKind::Month; return true; } - if (ParserKeyword("QUARTER").ignore(pos, expected) || ParserKeyword("SQL_TSI_QUARTER").ignore(pos, expected) + if (ParserKeyword("QUARTER").ignore(pos, expected) || ParserKeyword("QUARTERS").ignore(pos, expected) + || ParserKeyword("SQL_TSI_QUARTER").ignore(pos, expected) || ParserKeyword("QQ").ignore(pos, expected) || ParserKeyword("Q").ignore(pos, expected)) { result = IntervalKind::Quarter; return true; } - if (ParserKeyword("YEAR").ignore(pos, expected) || ParserKeyword("SQL_TSI_YEAR").ignore(pos, expected) + if (ParserKeyword("YEAR").ignore(pos, expected) || ParserKeyword("YEARS").ignore(pos, expected) + || ParserKeyword("SQL_TSI_YEAR").ignore(pos, expected) || ParserKeyword("YYYY").ignore(pos, expected) || ParserKeyword("YY").ignore(pos, expected)) { result = IntervalKind::Year; diff --git a/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.reference b/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.reference new file mode 100644 index 00000000000..4f1d0bdcd49 --- /dev/null +++ b/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.reference @@ -0,0 +1,16 @@ +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2009-02-14 01:31:30 +2009-02-14 01:31:30 +2009-02-15 23:31:30 +2009-02-15 23:31:30 diff --git a/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.sql b/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.sql new file mode 100644 index 00000000000..dd5fc9eca45 --- /dev/null +++ b/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.sql @@ -0,0 +1,16 @@ +SELECT INTERVAL 2 year; +SELECT INTERVAL 2 years; +SELECT INTERVAL 2 month; +SELECT INTERVAL 2 months; +SELECT INTERVAL 2 week; +SELECT INTERVAL 2 weeks; +SELECT INTERVAL 2 day; +SELECT INTERVAL 2 days; +SELECT INTERVAL 2 hour; +SELECT INTERVAL 2 hours; +SELECT INTERVAL 2 minute; +SELECT INTERVAL 2 minutes; +SELECT DATE_ADD(hour, 2, toDateTime(1234567890, 'UTC')); +SELECT DATE_ADD(hours, 2, toDateTime(1234567890, 'UTC')); +SELECT DATE_ADD(toDateTime(1234567890, 'UTC'), INTERVAL 2 day); +SELECT DATE_ADD(toDateTime(1234567890, 'UTC'), INTERVAL 2 days); From 0518931bbd1e76e195fd798fa6d66a3a9f828cc3 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Wed, 20 Sep 2023 23:16:36 +0200 Subject: [PATCH 63/69] Add more tests --- ...2884_interval_operator_support_plural_literal.reference | 7 +++++++ .../02884_interval_operator_support_plural_literal.sql | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.reference b/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.reference index 4f1d0bdcd49..9616b4c1415 100644 --- a/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.reference +++ b/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.reference @@ -10,7 +10,14 @@ 2 2 2 +2 +2 +2 +2 +2 +2 2009-02-14 01:31:30 2009-02-14 01:31:30 2009-02-15 23:31:30 2009-02-15 23:31:30 +2009-02-15 23:31:30 diff --git a/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.sql b/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.sql index dd5fc9eca45..41403cdf72e 100644 --- a/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.sql +++ b/tests/queries/0_stateless/02884_interval_operator_support_plural_literal.sql @@ -1,16 +1,23 @@ SELECT INTERVAL 2 year; SELECT INTERVAL 2 years; +SELECT INTERVAL '2 years'; SELECT INTERVAL 2 month; SELECT INTERVAL 2 months; +SELECT INTERVAL '2 months'; SELECT INTERVAL 2 week; SELECT INTERVAL 2 weeks; +SELECT INTERVAL '2 weeks'; SELECT INTERVAL 2 day; SELECT INTERVAL 2 days; +SELECT INTERVAL '2 days'; SELECT INTERVAL 2 hour; SELECT INTERVAL 2 hours; +SELECT INTERVAL '2 hours'; SELECT INTERVAL 2 minute; SELECT INTERVAL 2 minutes; +SELECT INTERVAL '2 minutes'; SELECT DATE_ADD(hour, 2, toDateTime(1234567890, 'UTC')); SELECT DATE_ADD(hours, 2, toDateTime(1234567890, 'UTC')); SELECT DATE_ADD(toDateTime(1234567890, 'UTC'), INTERVAL 2 day); SELECT DATE_ADD(toDateTime(1234567890, 'UTC'), INTERVAL 2 days); +SELECT DATE_ADD(toDateTime(1234567890, 'UTC'), INTERVAL '2 days'); From 4a2f7976f01024154e61a99c6d9628e06816b211 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 20 Sep 2023 19:43:02 -0400 Subject: [PATCH 64/69] Resolve PR issues --- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageS3.h | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 436f7a3c84a..844e7310b16 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1071,7 +1071,6 @@ Pipe StorageS3::read( size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); num_streams = std::min(num_streams, estimated_keys_count); - LOG_INFO(&Poco::Logger::get("StorageS3"), "adjusting num_streams={}", num_streams); auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) @@ -1079,6 +1078,7 @@ Pipe StorageS3::read( const size_t max_threads = local_context->getSettingsRef().max_threads; const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / num_streams); + LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Reading in {} streams, {} threads per stream", num_streams, max_parsing_threads); pipes.reserve(num_streams); for (size_t i = 0; i < num_streams; ++i) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index f0315244088..088f9000ce8 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -62,6 +62,8 @@ public: /// Estimates how many streams we need to process all files. /// If keys count >= max_threads_count, the returned number may not represent the actual number of the keys. + /// Intended to be called before any next() calls, may underestimate otherwise + /// fixme: May underestimate if the glob has a strong filter, so there are few matches among the first 1000 ListObjects results. virtual size_t estimatedKeysCount() = 0; KeyWithInfo operator ()() { return next(); } From 2901eab8ccf16955b7040f7ac729f14d9d390e04 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 21 Sep 2023 03:44:51 +0000 Subject: [PATCH 65/69] avoid race condition on last_updated_time Signed-off-by: Duc Canh Le --- src/Common/isLocalAddress.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index ce0316db2c4..902505404a6 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -91,16 +91,19 @@ struct NetworkInterfaces : public boost::noncopyable { static constexpr int NET_INTERFACE_VALID_PERIOD_MS = 30000; static NetworkInterfaces nf; - static auto last_updated_time = std::chrono::steady_clock::now(); + static std::atomic last_updated_time = std::chrono::steady_clock::now(); static std::shared_mutex nf_mtx; auto now = std::chrono::steady_clock::now(); + auto last_updated_time_snapshot = last_updated_time.load(); - if (std::chrono::duration_cast(now - last_updated_time).count() > NET_INTERFACE_VALID_PERIOD_MS) + if (std::chrono::duration_cast(now - last_updated_time_snapshot).count() > NET_INTERFACE_VALID_PERIOD_MS) { std::unique_lock lock(nf_mtx); + if (last_updated_time.load() != last_updated_time_snapshot) /// it's possible that last_updated_time after we get the snapshot + return nf; nf.swap(NetworkInterfaces()); - last_updated_time = now; + last_updated_time.store(now); return nf; } else From 7a8246ecded2161cc86df2084b8a05a1f9d7171a Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Thu, 21 Sep 2023 07:07:32 +0200 Subject: [PATCH 66/69] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a6e23e92d36..30ac45ced03 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -31,7 +31,7 @@ * Add new functions `structureToCapnProtoSchema`/`structureToProtobufSchema` that convert ClickHouse table structure to CapnProto/Protobuf format schema. Allow to input/output data in CapnProto/Protobuf format without external format schema using autogenerated schema from table structure (controled by settings `format_capn_proto_use_autogenerated_schema`/`format_protobuf_use_autogenerated_schema`). Allow to export autogenerated schema while input/outoput using setting `output_format_schema`. [#52278](https://github.com/ClickHouse/ClickHouse/pull/52278) ([Kruglov Pavel](https://github.com/Avogar)). * A new field `query_cache_usage` in `system.query_log` now shows if and how the query cache was used. [#52384](https://github.com/ClickHouse/ClickHouse/pull/52384) ([Robert Schulze](https://github.com/rschu1ze)). * Add new function `startsWithUTF8` and `endsWithUTF8`. [#52555](https://github.com/ClickHouse/ClickHouse/pull/52555) ([李扬](https://github.com/taiyang-li)). -* Allow variable number of columns in TSV/CustomSeprarated/JSONCompactEachRow, make schema inference work with variable number of columns. Add settings `input_format_tsv_allow_variable_number_of_columns`, `input_format_custom_allow_variable_number_of_columns`, `input_format_json_compact_allow_variable_number_of_columns`. [#52692](https://github.com/ClickHouse/ClickHouse/pull/52692) ([Kruglov Pavel](https://github.com/Avogar)). +* Allow variable number of columns in TSV/CustomSeparated/JSONCompactEachRow, make schema inference work with variable number of columns. Add settings `input_format_tsv_allow_variable_number_of_columns`, `input_format_custom_allow_variable_number_of_columns`, `input_format_json_compact_allow_variable_number_of_columns`. [#52692](https://github.com/ClickHouse/ClickHouse/pull/52692) ([Kruglov Pavel](https://github.com/Avogar)). * Added `SYSTEM STOP/START PULLING REPLICATION LOG` queries (for testing `ReplicatedMergeTree`). [#52881](https://github.com/ClickHouse/ClickHouse/pull/52881) ([Alexander Tokmakov](https://github.com/tavplubix)). * Allow to execute constant non-deterministic functions in mutations on initiator. [#53129](https://github.com/ClickHouse/ClickHouse/pull/53129) ([Anton Popov](https://github.com/CurtizJ)). * Add input format `One` that doesn't read any data and always returns single row with column `dummy` with type `UInt8` and value `0` like `system.one`. It can be used together with `_file/_path` virtual columns to list files in file/s3/url/hdfs/etc table functions without reading any data. [#53209](https://github.com/ClickHouse/ClickHouse/pull/53209) ([Kruglov Pavel](https://github.com/Avogar)). From a0862531cbaea2b088668cbcd2f3238439e00dd8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 21 Sep 2023 09:53:52 +0000 Subject: [PATCH 67/69] Fix clang-17 build --- src/Server/KeeperTCPHandler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 84ed7388503..742300f9b2e 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -382,9 +382,9 @@ void KeeperTCPHandler::runImpl() } auto response_fd = poll_wrapper->getResponseFD(); - auto response_callback = [responses = this->responses, response_fd](const Coordination::ZooKeeperResponsePtr & response) + auto response_callback = [responses_ = this->responses, response_fd](const Coordination::ZooKeeperResponsePtr & response) { - if (!responses->push(response)) + if (!responses_->push(response)) throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push response with xid {} and zxid {}", response->xid, From 853e3f0aa789d5b6dcb251a403276d9fdc02902c Mon Sep 17 00:00:00 2001 From: Pavel Novitskiy <106629323+pnovitskiy@users.noreply.github.com> Date: Thu, 21 Sep 2023 15:20:18 +0200 Subject: [PATCH 68/69] Add drop table IF EMPTY (#48915) --- docs/en/sql-reference/statements/drop.md | 3 +- src/Common/ErrorCodes.cpp | 2 +- src/Interpreters/InterpreterDropQuery.cpp | 38 ++++++- src/Parsers/ASTDropQuery.cpp | 3 + src/Parsers/ASTDropQuery.h | 1 + src/Parsers/ParserDropQuery.cpp | 9 ++ src/Parsers/ParserDropQuery.h | 5 +- .../test_drop_if_empty/__init__.py | 0 .../configs/drop_if_empty_check.xml | 14 +++ .../configs/remote_servers.xml | 17 +++ .../test_drop_if_empty/configs/zookeeper.xml | 19 ++++ tests/integration/test_drop_if_empty/test.py | 105 ++++++++++++++++++ .../0_stateless/02716_drop_if_empty.reference | 10 ++ .../0_stateless/02716_drop_if_empty.sql | 22 ++++ 14 files changed, 243 insertions(+), 5 deletions(-) create mode 100644 tests/integration/test_drop_if_empty/__init__.py create mode 100644 tests/integration/test_drop_if_empty/configs/drop_if_empty_check.xml create mode 100644 tests/integration/test_drop_if_empty/configs/remote_servers.xml create mode 100644 tests/integration/test_drop_if_empty/configs/zookeeper.xml create mode 100644 tests/integration/test_drop_if_empty/test.py create mode 100644 tests/queries/0_stateless/02716_drop_if_empty.reference create mode 100644 tests/queries/0_stateless/02716_drop_if_empty.sql diff --git a/docs/en/sql-reference/statements/drop.md b/docs/en/sql-reference/statements/drop.md index eed76dbcd5c..8ed00f625d6 100644 --- a/docs/en/sql-reference/statements/drop.md +++ b/docs/en/sql-reference/statements/drop.md @@ -21,6 +21,7 @@ DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] [SYNC] ## DROP TABLE Deletes the table. +In case when `IF EMPTY` clause is specified server will check if table is empty only on replica that received initial query. :::tip Also see [UNDROP TABLE](/docs/en/sql-reference/statements/undrop.md) @@ -29,7 +30,7 @@ Also see [UNDROP TABLE](/docs/en/sql-reference/statements/undrop.md) Syntax: ``` sql -DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] [SYNC] +DROP [TEMPORARY] TABLE [IF EXISTS] [IF EMPTY] [db.]name [ON CLUSTER cluster] [SYNC] ``` ## DROP DICTIONARY diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index be2b0a7bd5e..a4837ac3c44 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -587,7 +587,7 @@ M(702, CLIENT_INFO_DOES_NOT_MATCH) \ M(703, INVALID_IDENTIFIER) \ M(704, CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS) \ - \ + M(705, TABLE_NOT_EMPTY) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 9dd28cbeaac..dd52b6c2e14 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -34,6 +34,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int INCORRECT_QUERY; extern const int TABLE_IS_READ_ONLY; + extern const int TABLE_NOT_EMPTY; } namespace ActionLocks @@ -55,7 +56,8 @@ InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, ContextMut BlockIO InterpreterDropQuery::execute() { auto & drop = query_ptr->as(); - if (!drop.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext())) + + if (!drop.cluster.empty() && drop.table && !drop.if_empty && !maybeRemoveOnCluster(query_ptr, getContext())) { DDLQueryOnClusterParams params; params.access_to_check = getRequiredAccessForDDLOnCluster(); @@ -67,6 +69,12 @@ BlockIO InterpreterDropQuery::execute() if (drop.table) return executeToTable(drop); + else if (drop.database && !drop.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext())) + { + DDLQueryOnClusterParams params; + params.access_to_check = getRequiredAccessForDDLOnCluster(); + return executeDDLQueryOnCluster(query_ptr, getContext(), params); + } else if (drop.database) return executeToDatabase(drop); else @@ -122,6 +130,12 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue if (database && table) { + const auto & settings = getContext()->getSettingsRef(); + if (query.if_empty) + { + if (auto rows = table->totalRows(settings); rows > 0) + throw Exception(ErrorCodes::TABLE_NOT_EMPTY, "Table {} is not empty", backQuoteIfNeed(table_id.table_name)); + } checkStorageSupportsTransactionsIfNeeded(table, context_); auto & ast_drop_query = query.as(); @@ -151,6 +165,18 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue else drop_storage = AccessType::DROP_TABLE; + auto new_query_ptr = query.clone(); + auto & query_to_send = new_query_ptr->as(); + + if (!query.cluster.empty() && !maybeRemoveOnCluster(new_query_ptr, getContext())) + { + query_to_send.if_empty = false; + + DDLQueryOnClusterParams params; + params.access_to_check = getRequiredAccessForDDLOnCluster(); + return executeDDLQueryOnCluster(new_query_ptr, getContext(), params); + } + if (database->shouldReplicateQuery(getContext(), query_ptr)) { if (query.kind == ASTDropQuery::Kind::Detach) @@ -162,7 +188,10 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue ddl_guard->releaseTableLock(); table.reset(); - return database->tryEnqueueReplicatedDDL(query.clone(), context_); + + query_to_send.if_empty = false; + + return database->tryEnqueueReplicatedDDL(new_query_ptr, context_); } if (query.kind == ASTDropQuery::Kind::Detach) @@ -340,9 +369,13 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, if (query.kind == ASTDropQuery::Kind::Detach && query.permanently) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DETACH PERMANENTLY is not implemented for databases"); + if (query.if_empty) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DROP IF EMPTY is not implemented for databases"); + if (database->hasReplicationThread()) database->stopReplication(); + if (database->shouldBeEmptyOnDetach()) { /// Cancel restarting replicas in that database, wait for remaining RESTART queries to finish. @@ -355,6 +388,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, if (truncate) query_for_table.kind = ASTDropQuery::Kind::Drop; query_for_table.if_exists = true; + query_for_table.if_empty = false; query_for_table.setDatabase(database_name); query_for_table.sync = query.sync; diff --git a/src/Parsers/ASTDropQuery.cpp b/src/Parsers/ASTDropQuery.cpp index 93a4b547025..ad1294c6e71 100644 --- a/src/Parsers/ASTDropQuery.cpp +++ b/src/Parsers/ASTDropQuery.cpp @@ -60,6 +60,9 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState if (if_exists) settings.ostr << "IF EXISTS "; + if (if_empty) + settings.ostr << "IF EMPTY "; + settings.ostr << (settings.hilite ? hilite_none : ""); if (!table && database) diff --git a/src/Parsers/ASTDropQuery.h b/src/Parsers/ASTDropQuery.h index 05515ba4005..a732b354260 100644 --- a/src/Parsers/ASTDropQuery.h +++ b/src/Parsers/ASTDropQuery.h @@ -21,6 +21,7 @@ public: Kind kind; bool if_exists{false}; + bool if_empty{false}; /// Useful if we already have a DDL lock bool no_ddl_lock{false}; diff --git a/src/Parsers/ParserDropQuery.cpp b/src/Parsers/ParserDropQuery.cpp index f40a39e6b2f..450c8a1afec 100644 --- a/src/Parsers/ParserDropQuery.cpp +++ b/src/Parsers/ParserDropQuery.cpp @@ -19,6 +19,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons ParserKeyword s_database("DATABASE"); ParserToken s_dot(TokenType::Dot); ParserKeyword s_if_exists("IF EXISTS"); + ParserKeyword s_if_empty("IF EMPTY"); ParserIdentifier name_p(true); ParserKeyword s_permanently("PERMANENTLY"); ParserKeyword s_no_delay("NO DELAY"); @@ -28,6 +29,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons ASTPtr table; String cluster_str; bool if_exists = false; + bool if_empty = false; bool temporary = false; bool is_dictionary = false; bool is_view = false; @@ -39,6 +41,9 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons if (s_if_exists.ignore(pos, expected)) if_exists = true; + if (s_if_empty.ignore(pos, expected)) + if_empty = true; + if (!name_p.parse(pos, database, expected)) return false; } @@ -60,6 +65,9 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons if (s_if_exists.ignore(pos, expected)) if_exists = true; + if (s_if_empty.ignore(pos, expected)) + if_empty = true; + if (!name_p.parse(pos, table, expected)) return false; @@ -90,6 +98,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons query->kind = kind; query->if_exists = if_exists; + query->if_empty = if_empty; query->temporary = temporary; query->is_dictionary = is_dictionary; query->is_view = is_view; diff --git a/src/Parsers/ParserDropQuery.h b/src/Parsers/ParserDropQuery.h index 39ff5b7c4fd..f77b477a2fd 100644 --- a/src/Parsers/ParserDropQuery.h +++ b/src/Parsers/ParserDropQuery.h @@ -8,7 +8,10 @@ namespace DB { /** Query like this: - * DROP|DETACH|TRUNCATE TABLE [IF EXISTS] [db.]name [PERMANENTLY] + * DROP TABLE [IF EXISTS|EMPTY] [db.]name [PERMANENTLY] + * + * Or: + * DETACH|TRUNCATE TABLE [IF EXISTS] [db.]name [PERMANENTLY] * * Or: * DROP DATABASE [IF EXISTS] db diff --git a/tests/integration/test_drop_if_empty/__init__.py b/tests/integration/test_drop_if_empty/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_drop_if_empty/configs/drop_if_empty_check.xml b/tests/integration/test_drop_if_empty/configs/drop_if_empty_check.xml new file mode 100644 index 00000000000..34a78ac9919 --- /dev/null +++ b/tests/integration/test_drop_if_empty/configs/drop_if_empty_check.xml @@ -0,0 +1,14 @@ + + 8123 + 9000 + + + 1 + + + + + default + + + diff --git a/tests/integration/test_drop_if_empty/configs/remote_servers.xml b/tests/integration/test_drop_if_empty/configs/remote_servers.xml new file mode 100644 index 00000000000..34fda471187 --- /dev/null +++ b/tests/integration/test_drop_if_empty/configs/remote_servers.xml @@ -0,0 +1,17 @@ + + + + + + node1 + 9000 + + + node2 + 9000 + + + + + + diff --git a/tests/integration/test_drop_if_empty/configs/zookeeper.xml b/tests/integration/test_drop_if_empty/configs/zookeeper.xml new file mode 100644 index 00000000000..d23bf1625ce --- /dev/null +++ b/tests/integration/test_drop_if_empty/configs/zookeeper.xml @@ -0,0 +1,19 @@ + + + + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + 20000 + + + diff --git a/tests/integration/test_drop_if_empty/test.py b/tests/integration/test_drop_if_empty/test.py new file mode 100644 index 00000000000..d96936eb826 --- /dev/null +++ b/tests/integration/test_drop_if_empty/test.py @@ -0,0 +1,105 @@ +import re +import time + +import pytest +import requests +from helpers.cluster import ClickHouseCluster +from helpers.client import QueryRuntimeException + +cluster = ClickHouseCluster(__file__, zookeeper_config_path="configs/zookeeper.xml") + +main_configs = [ + "configs/remote_servers.xml", +] + +node1 = cluster.add_instance( + "node1", + main_configs=main_configs + ["configs/drop_if_empty_check.xml"], + macros={"replica": "node1", "shard": "shard1"}, + with_zookeeper=True, +) + +node2 = cluster.add_instance( + "node2", + main_configs=main_configs + ["configs/drop_if_empty_check.xml"], + macros={"replica": "node2", "shard": "shard1"}, + with_zookeeper=True, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_drop_if_empty(start_cluster): + settings = { + "allow_experimental_database_replicated": 1, + } + node1.query( + "CREATE DATABASE replicateddb " + "ENGINE = Replicated('/clickhouse/databases/replicateddb', 'shard1', 'node1')", + settings=settings, + ) + node2.query( + "CREATE DATABASE replicateddb " + "ENGINE = Replicated('/clickhouse/databases/replicateddb', 'shard1', 'node2')", + settings=settings, + ) + node1.query( + "CREATE TABLE default.tbl ON CLUSTER 'cluster' (" + "x UInt64" + ") ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}')" + "ORDER BY x" + ) + node1.query( + "CREATE TABLE replicateddb.tbl2 (" "x UInt64" ") ENGINE=MergeTree " "ORDER BY x" + ) + + assert 1 == int( + node2.query("SELECT count() FROM system.tables WHERE name = 'tbl';") + ) + assert 1 == int( + node2.query("SELECT count() FROM system.databases WHERE name = 'replicateddb';") + ) + assert 1 == int( + node2.query("SELECT count() FROM system.tables WHERE name = 'tbl2';") + ) + + node2.query("SYSTEM STOP MERGES;") + node2.query("SYSTEM STOP FETCHES;") + node2.query("SYSTEM STOP REPLICATION QUEUES;") + + node1.query("INSERT INTO default.tbl SELECT * FROM system.numbers_mt LIMIT 10000;") + node1.query( + "INSERT INTO replicateddb.tbl2 SELECT * FROM system.numbers_mt LIMIT 10000;" + ) + + assert 0 == int(node2.query("SELECT count() FROM default.tbl;")) + assert 0 == int(node2.query("SELECT count() FROM replicateddb.tbl2;")) + + node2.query("DROP TABLE IF EMPTY default.tbl ON CLUSTER 'cluster';") + node2.query("DROP TABLE IF EMPTY replicateddb.tbl2;") + + assert 0 == int( + node1.query("SELECT count() FROM system.tables WHERE name = 'tbl';") + ) + assert 0 == int( + node2.query("SELECT count() FROM system.tables WHERE name = 'tbl';") + ) + assert 0 == int( + node1.query("SELECT count() FROM system.tables WHERE name = 'tbl2';") + ) + assert 0 == int( + node2.query("SELECT count() FROM system.tables WHERE name = 'tbl2';") + ) + + with pytest.raises( + QueryRuntimeException, + match="DB::Exception: DROP IF EMPTY is not implemented for databases.", + ): + node2.query("DROP DATABASE IF EMPTY replicateddb;") diff --git a/tests/queries/0_stateless/02716_drop_if_empty.reference b/tests/queries/0_stateless/02716_drop_if_empty.reference new file mode 100644 index 00000000000..6ddd9b205af --- /dev/null +++ b/tests/queries/0_stateless/02716_drop_if_empty.reference @@ -0,0 +1,10 @@ +-- { echoOn } +DROP TABLE IF EMPTY data_02716_2; +DROP TABLE IF EMPTY data_02716_1; -- { serverError TABLE_NOT_EMPTY } +TRUNCATE TABLE data_02716_1; +DROP TABLE IF EMPTY data_02716_1; +DROP DATABASE IF EMPTY {CLICKHOUSE_DATABASE_1:Identifier}; -- { serverError NOT_IMPLEMENTED } +SELECT count() FROM system.tables WHERE database = {CLICKHOUSE_DATABASE_1:String}; +1 +SELECT count() FROM system.tables WHERE database = 'default' AND name IN ('data_02716_1', 'data_02716_2'); +0 diff --git a/tests/queries/0_stateless/02716_drop_if_empty.sql b/tests/queries/0_stateless/02716_drop_if_empty.sql new file mode 100644 index 00000000000..6cea90a6d5c --- /dev/null +++ b/tests/queries/0_stateless/02716_drop_if_empty.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS data_02716_1; +DROP TABLE IF EXISTS data_02716_2; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}.data_02716_3; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; + +CREATE TABLE data_02716_1 (v UInt64) ENGINE = MergeTree ORDER BY v; +CREATE TABLE data_02716_2 (v UInt64) ENGINE = MergeTree ORDER BY v; + +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; +CREATE TABLE {CLICKHOUSE_DATABASE_1:Identifier}.data_02716_3 (v UInt64) ENGINE = MergeTree ORDER BY v; + +INSERT INTO data_02716_1 SELECT * FROM system.numbers LIMIT 1; + +-- { echoOn } +DROP TABLE IF EMPTY data_02716_2; +DROP TABLE IF EMPTY data_02716_1; -- { serverError TABLE_NOT_EMPTY } +TRUNCATE TABLE data_02716_1; +DROP TABLE IF EMPTY data_02716_1; +DROP DATABASE IF EMPTY {CLICKHOUSE_DATABASE_1:Identifier}; -- { serverError NOT_IMPLEMENTED } + +SELECT count() FROM system.tables WHERE database = {CLICKHOUSE_DATABASE_1:String}; +SELECT count() FROM system.tables WHERE database = 'default' AND name IN ('data_02716_1', 'data_02716_2'); From 8610baabefbe0e90fa861ea9e86c568bad09879b Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Thu, 21 Sep 2023 15:57:47 +0200 Subject: [PATCH 69/69] Revert "Avoid excessive calls to getifaddrs in isLocalAddress" --- src/Common/isLocalAddress.cpp | 43 +++-------------------------------- 1 file changed, 3 insertions(+), 40 deletions(-) diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index 902505404a6..7569c6fc14e 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -1,14 +1,9 @@ #include #include -#include #include -#include -#include -#include #include #include -#include #include #include #include @@ -25,7 +20,7 @@ namespace ErrorCodes namespace { -struct NetworkInterfaces : public boost::noncopyable +struct NetworkInterfaces { ifaddrs * ifaddr; NetworkInterfaces() @@ -36,13 +31,6 @@ struct NetworkInterfaces : public boost::noncopyable } } - void swap(NetworkInterfaces && other) - { - auto * tmp = ifaddr; - ifaddr = other.ifaddr; - other.ifaddr = tmp; - } - bool hasAddress(const Poco::Net::IPAddress & address) const { ifaddrs * iface; @@ -86,32 +74,6 @@ struct NetworkInterfaces : public boost::noncopyable { freeifaddrs(ifaddr); } - - static const NetworkInterfaces & instance() - { - static constexpr int NET_INTERFACE_VALID_PERIOD_MS = 30000; - static NetworkInterfaces nf; - static std::atomic last_updated_time = std::chrono::steady_clock::now(); - static std::shared_mutex nf_mtx; - - auto now = std::chrono::steady_clock::now(); - auto last_updated_time_snapshot = last_updated_time.load(); - - if (std::chrono::duration_cast(now - last_updated_time_snapshot).count() > NET_INTERFACE_VALID_PERIOD_MS) - { - std::unique_lock lock(nf_mtx); - if (last_updated_time.load() != last_updated_time_snapshot) /// it's possible that last_updated_time after we get the snapshot - return nf; - nf.swap(NetworkInterfaces()); - last_updated_time.store(now); - return nf; - } - else - { - std::shared_lock lock(nf_mtx); - return nf; - } - } }; } @@ -149,7 +111,8 @@ bool isLocalAddress(const Poco::Net::IPAddress & address) } } - return NetworkInterfaces::instance().hasAddress(address); + NetworkInterfaces interfaces; + return interfaces.hasAddress(address); }