From 995b51ef736c20d9490f75cebaa4f42291df40cf Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 7 Dec 2023 03:41:32 +0000 Subject: [PATCH 01/61] Allow avoiding resolving hostnames in DDLWorker --- programs/server/config.xml | 3 + src/Interpreters/DDLTask.cpp | 18 ++++- src/Interpreters/DDLTask.h | 2 +- src/Interpreters/DDLWorker.cpp | 5 +- src/Interpreters/DDLWorker.h | 2 + .../test_ddl_config_hostname/__init__.py | 0 .../configs/remote_servers.xml | 19 +++++ .../test_ddl_config_hostname/test.py | 80 +++++++++++++++++++ 8 files changed, 125 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_ddl_config_hostname/__init__.py create mode 100644 tests/integration/test_ddl_config_hostname/configs/remote_servers.xml create mode 100644 tests/integration/test_ddl_config_hostname/test.py diff --git a/programs/server/config.xml b/programs/server/config.xml index e333082d099..688f0bf5645 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1380,6 +1380,9 @@ + + + diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 6e9155ab2a2..172d68f2941 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -215,14 +215,28 @@ ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const Z } -bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper) +bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper, const std::optional & config_host_name) { bool host_in_hostlist = false; std::exception_ptr first_exception = nullptr; + auto maybe_secure_port = global_context->getTCPPortSecure(); + for (const HostID & host : entry.hosts) { - auto maybe_secure_port = global_context->getTCPPortSecure(); + if (config_host_name) + { + if (host.host_name != *config_host_name) + continue; + + if (!(maybe_secure_port && maybe_secure_port == host.port) && !(global_context->getTCPPort() == host.port)) + continue; + + host_in_hostlist = true; + host_id = host; + host_id_str = host.toString(); + break; + } try { diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index 1ceb74c7048..e1a81ac97af 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -143,7 +143,7 @@ struct DDLTask : public DDLTaskBase { DDLTask(const String & name, const String & path) : DDLTaskBase(name, path) {} - bool findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper); + bool findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper, const std::optional & config_host_name); void setClusterInfo(ContextPtr context, Poco::Logger * log); diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 30cf6fd0568..de24dea1857 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -107,6 +107,9 @@ DDLWorker::DDLWorker( cleanup_delay_period = config->getUInt64(prefix + ".cleanup_delay_period", static_cast(cleanup_delay_period)); max_tasks_in_queue = std::max(1, config->getUInt64(prefix + ".max_tasks_in_queue", max_tasks_in_queue)); + if (config->has(prefix + ".host_name")) + config_host_name = config->getString(prefix + ".host_name"); + if (config->has(prefix + ".profile")) context->setSetting("profile", config->getString(prefix + ".profile")); } @@ -214,7 +217,7 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r /// Stage 2: resolve host_id and check if we should execute query or not /// Multiple clusters can use single DDL queue path in ZooKeeper, /// So we should skip task if we cannot find current host in cluster hosts list. - if (!task->findCurrentHostID(context, log, zookeeper)) + if (!task->findCurrentHostID(context, log, zookeeper, config_host_name)) { out_reason = "There is no a local address in host list"; return add_to_skip_set(); diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index d34a4135199..adc9a491d81 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -153,6 +153,8 @@ protected: ContextMutablePtr context; Poco::Logger * log; + std::optional config_host_name; /// host_name from config + std::string host_fqdn; /// current host domain name std::string host_fqdn_id; /// host_name:port std::string queue_dir; /// dir with queue of queries diff --git a/tests/integration/test_ddl_config_hostname/__init__.py b/tests/integration/test_ddl_config_hostname/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_ddl_config_hostname/configs/remote_servers.xml b/tests/integration/test_ddl_config_hostname/configs/remote_servers.xml new file mode 100644 index 00000000000..8c6a507951d --- /dev/null +++ b/tests/integration/test_ddl_config_hostname/configs/remote_servers.xml @@ -0,0 +1,19 @@ + + + + + true + + node1 + 9000 + + + + + + 1 + + + node1 + + diff --git a/tests/integration/test_ddl_config_hostname/test.py b/tests/integration/test_ddl_config_hostname/test.py new file mode 100644 index 00000000000..f6cb5f5c38e --- /dev/null +++ b/tests/integration/test_ddl_config_hostname/test.py @@ -0,0 +1,80 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", + main_configs=["configs/remote_servers.xml"], + with_zookeeper=True, + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_ddl_queue_delete_add_replica(started_cluster): + # Some query started on the cluster, then we deleted some unfinished node + # and added a new node to the cluster. Considering that there are less + # finished nodes than expected and we can't resolve deleted node's hostname + # the queue will be stuck on a new node. + # inside allows us to simply discard deleted + # node's hostname by simple comparison without trying to resolve it. + + node1.query( + "create table hostname_change on cluster test_cluster (n int) engine=Log" + ) + + # There's no easy way to change hostname of a container, so let's update values in zk + query_znode = node1.query( + "select max(name) from system.zookeeper where path='/clickhouse/task_queue/ddl'" + )[:-1] + + value = ( + node1.query( + "select value from system.zookeeper where path='/clickhouse/task_queue/ddl' and name='{}' format TSVRaw".format( + query_znode + ) + )[:-1] + .replace("hosts: ['node1:9000']", "hosts: ['finished_node:9000','deleted_node:9000']") + .replace("initiator: node1:9000", "initiator: finished_node:9000") + .replace("\\'", "#") + .replace("'", "\\'") + .replace("\n", "\\n") + .replace("#", "\\'") + ) + + finished_znode = node1.query( + "select name from system.zookeeper where path='/clickhouse/task_queue/ddl/{}/finished' and name like '%node1%'".format( + query_znode + ) + )[:-1] + + node1.query( + "insert into system.zookeeper (name, path, value) values ('{}', '/clickhouse/task_queue/ddl', '{}')".format( + query_znode, value + ) + ) + started_cluster.get_kazoo_client("zoo1").delete( + "/clickhouse/task_queue/ddl/{}/finished/{}".format(query_znode, finished_znode) + ) + + node1.query( + "insert into system.zookeeper (name, path, value) values ('{}', '/clickhouse/task_queue/ddl/{}/finished', '0\\n')".format( + finished_znode.replace("node1", "finished_node"), query_znode + ) + ) + + node1.restart_clickhouse(kill=True) + + node1.query( + "create table hostname_change2 on cluster test_cluster (n int) engine=Log" + ) From 10bfd054d82bd70d0a931ad71c6031a54306740c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 8 Dec 2023 19:40:09 +0000 Subject: [PATCH 02/61] Fix style & review --- src/Interpreters/DDLTask.cpp | 4 ++-- .../test_ddl_config_hostname/test.py | 24 ++++++++----------- 2 files changed, 12 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 172d68f2941..0164f5668a2 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -226,10 +226,10 @@ bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, c { if (config_host_name) { - if (host.host_name != *config_host_name) + if (config_host_name != host.host_name) continue; - if (!(maybe_secure_port && maybe_secure_port == host.port) && !(global_context->getTCPPort() == host.port)) + if (maybe_secure_port != host.port && global_context->getTCPPort() != host.port) continue; host_in_hostlist = true; diff --git a/tests/integration/test_ddl_config_hostname/test.py b/tests/integration/test_ddl_config_hostname/test.py index f6cb5f5c38e..724e766c9dc 100644 --- a/tests/integration/test_ddl_config_hostname/test.py +++ b/tests/integration/test_ddl_config_hostname/test.py @@ -40,11 +40,11 @@ def test_ddl_queue_delete_add_replica(started_cluster): value = ( node1.query( - "select value from system.zookeeper where path='/clickhouse/task_queue/ddl' and name='{}' format TSVRaw".format( - query_znode - ) + f"select value from system.zookeeper where path='/clickhouse/task_queue/ddl' and name='{query_znode}' format TSVRaw" )[:-1] - .replace("hosts: ['node1:9000']", "hosts: ['finished_node:9000','deleted_node:9000']") + .replace( + "hosts: ['node1:9000']", "hosts: ['finished_node:9000','deleted_node:9000']" + ) .replace("initiator: node1:9000", "initiator: finished_node:9000") .replace("\\'", "#") .replace("'", "\\'") @@ -53,24 +53,20 @@ def test_ddl_queue_delete_add_replica(started_cluster): ) finished_znode = node1.query( - "select name from system.zookeeper where path='/clickhouse/task_queue/ddl/{}/finished' and name like '%node1%'".format( - query_znode - ) + f"select name from system.zookeeper where path='/clickhouse/task_queue/ddl/{query_znode}/finished' and name like '%node1%'" )[:-1] node1.query( - "insert into system.zookeeper (name, path, value) values ('{}', '/clickhouse/task_queue/ddl', '{}')".format( - query_znode, value - ) + f"insert into system.zookeeper (name, path, value) values ('{query_znode}', '/clickhouse/task_queue/ddl', '{value}')" ) started_cluster.get_kazoo_client("zoo1").delete( - "/clickhouse/task_queue/ddl/{}/finished/{}".format(query_znode, finished_znode) + f"/clickhouse/task_queue/ddl/{query_znode}/finished/{finished_znode}" ) + finished_znode = finished_znode.replace("node1", "finished_node") + node1.query( - "insert into system.zookeeper (name, path, value) values ('{}', '/clickhouse/task_queue/ddl/{}/finished', '0\\n')".format( - finished_znode.replace("node1", "finished_node"), query_znode - ) + f"insert into system.zookeeper (name, path, value) values ('{finished_znode}', '/clickhouse/task_queue/ddl/{query_znode}/finished', '0\\n')" ) node1.restart_clickhouse(kill=True) From 547e3ed6c04a400222c7cab6205a1912c7d41760 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 26 Dec 2023 22:57:31 +0000 Subject: [PATCH 03/61] Add a check for the 'host_name' parameter --- src/Interpreters/DDLTask.cpp | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 0164f5668a2..d386ab9a91d 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -220,7 +220,20 @@ bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, c bool host_in_hostlist = false; std::exception_ptr first_exception = nullptr; - auto maybe_secure_port = global_context->getTCPPortSecure(); + const auto maybe_secure_port = global_context->getTCPPortSecure(); + const auto port = global_context->getTCPPort() + + if (config_host_name) + { + bool is_local_port = (maybe_secure_port && HostID(*config_host_name, *maybe_secure_port).isLocalAddress(*maybe_secure_port)) || + HostID(*config_host_name, port).isLocalAddress(port); + + if (!is_local_port) + throw Exception( + ErrorCodes::DNS_ERROR, + "{} is not a local adress. Check parameter 'host_name' in the configuration", + *config_host_name) + } for (const HostID & host : entry.hosts) { @@ -229,7 +242,7 @@ bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, c if (config_host_name != host.host_name) continue; - if (maybe_secure_port != host.port && global_context->getTCPPort() != host.port) + if (maybe_secure_port != host.port && port != host.port) continue; host_in_hostlist = true; @@ -242,7 +255,7 @@ bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, c { /// The port is considered local if it matches TCP or TCP secure port that the server is listening. bool is_local_port - = (maybe_secure_port && host.isLocalAddress(*maybe_secure_port)) || host.isLocalAddress(global_context->getTCPPort()); + = (maybe_secure_port && host.isLocalAddress(*maybe_secure_port)) || host.isLocalAddress(port); if (!is_local_port) continue; From 3140f869cc05692d6c665b3525efb8b5cd8f0f16 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Wed, 27 Dec 2023 17:15:44 +0100 Subject: [PATCH 04/61] Fix typo --- src/Interpreters/DDLTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index d386ab9a91d..e7796c5d3a5 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -231,7 +231,7 @@ bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, c if (!is_local_port) throw Exception( ErrorCodes::DNS_ERROR, - "{} is not a local adress. Check parameter 'host_name' in the configuration", + "{} is not a local address. Check parameter 'host_name' in the configuration", *config_host_name) } From 87eb18eb748f58e3cfbfe96d03124f3b0e04b7ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 28 Dec 2023 01:02:03 +0100 Subject: [PATCH 05/61] Speed up numbers table function --- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 36 +++++++++++++++---- 1 file changed, 29 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 41690c1b132..a88203e0fca 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include namespace DB @@ -22,6 +23,27 @@ extern const int TOO_MANY_ROWS; namespace { +MULTITARGET_FUNCTION_AVX2_SSE42( + MULTITARGET_FUNCTION_HEADER(void), + iotaImpl, MULTITARGET_FUNCTION_BODY((UInt64 * begin, UInt64 count, UInt64 first_value) + { + for (UInt64 i = 0; i < count; i++) + *(begin + i) = first_value + i; + }) +) + +static void iota(UInt64 * begin, UInt64 count, UInt64 first_value) +{ +#if USE_MULTITARGET_CODE + if (isArchSupported(TargetArch::AVX2)) + return iotaImplAVX2(begin, count, first_value); + + if (isArchSupported(TargetArch::SSE42)) + return iotaImplSSE42(begin, count, first_value); +#endif + return iotaImpl(begin, count, first_value); +} + class NumbersSource : public ISource { public: @@ -43,8 +65,7 @@ protected: size_t curr = next; /// The local variable for some reason works faster (>20%) than member of class. UInt64 * pos = vec.data(); /// This also accelerates the code. UInt64 * end = &vec[block_size]; - while (pos < end) - *pos++ = curr++; + iota(pos, end - pos, curr); next += step; @@ -211,17 +232,18 @@ protected: { auto start_value_64 = static_cast(start_value); auto end_value_64 = static_cast(end_value); - while (start_value_64 < end_value_64) - *(pos++) = start_value_64++; + auto size = end_value_64 - start_value_64; + iota(pos, size, start_value_64); + pos += size; } }; if (can_provide > need) { UInt64 start_value = first_value(range) + cursor.offset_in_range; - UInt64 end_value = start_value + need; /// end_value will never overflow - while (start_value < end_value) - *(pos++) = start_value++; + /// end_value will never overflow + iota(pos, need, start_value); + pos += need; provided += need; cursor.offset_in_range += need; From 146de5b220d57fa53f1f9bf2c66742202140d807 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 5 Dec 2023 12:28:23 +0000 Subject: [PATCH 06/61] Ignore MVs with dropped target table during pushing to views --- .../Transforms/buildPushingToViewsChain.cpp | 6 +++++- ...ropped_target_table_no_exception.reference | 4 ++++ ...with_dropped_target_table_no_exception.sql | 20 +++++++++++++++++++ 3 files changed, 29 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.reference create mode 100644 tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.sql diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index f85dc28f4c7..b8aafe305a8 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -316,7 +316,11 @@ Chain buildPushingToViewsChain( type = QueryViewsLogElement::ViewType::MATERIALIZED; result_chain.addTableLock(lock); - StoragePtr inner_table = materialized_view->getTargetTable(); + StoragePtr inner_table = materialized_view->tryGetTargetTable(); + /// If target table was dropped, ignore this materialized view. + if (!inner_table) + continue; + auto inner_table_id = inner_table->getStorageID(); auto inner_metadata_snapshot = inner_table->getInMemoryMetadataPtr(); diff --git a/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.reference b/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.reference new file mode 100644 index 00000000000..8fb8a08e3f9 --- /dev/null +++ b/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.reference @@ -0,0 +1,4 @@ +42 +42 +42 +42 diff --git a/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.sql b/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.sql new file mode 100644 index 00000000000..744b2578617 --- /dev/null +++ b/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.sql @@ -0,0 +1,20 @@ +drop table if exists from_table; +drop table if exists to_table; +drop table if exists mv; + +create table from_table (x UInt32) engine=MergeTree order by x; +create table to_table (x UInt32) engine=MergeTree order by x; +create materialized view mv to to_table as select * from from_table; + +insert into from_table select 42; +select * from from_table; +select * from to_table; + +drop table to_table; + +insert into from_table select 42; +select * from from_table; + +drop table from_table; +drop view mv; + From e66701dd101da0f446eb9b5b52a9aa48aef42a89 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 28 Dec 2023 15:00:39 +0000 Subject: [PATCH 07/61] Add setting ignore_materialized_views_with_dropped_target_table --- src/Core/Settings.h | 1 + .../Transforms/buildPushingToViewsChain.cpp | 13 ++++++++++++- src/Storages/StorageMaterializedView.h | 1 + ..._race_condition_between_insert_and_droppin_mv.sh | 2 +- 4 files changed, 15 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d96b1b9fc10..9e485d88772 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -584,6 +584,7 @@ class IColumn; M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \ M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ + M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped taraget table during pushing to views", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index b8aafe305a8..ab9b3a80f12 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -39,6 +39,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int UNKNOWN_TABLE; } ThreadStatusesHolder::~ThreadStatusesHolder() @@ -319,7 +320,17 @@ Chain buildPushingToViewsChain( StoragePtr inner_table = materialized_view->tryGetTargetTable(); /// If target table was dropped, ignore this materialized view. if (!inner_table) - continue; + { + if (context->getSettingsRef().ignore_materialized_views_with_dropped_target_table) + continue; + + throw Exception( + ErrorCodes::UNKNOWN_TABLE, + "Target table '{}' of view '{}' doesn't exists. To ignore this view use setting " + "ignore_materialized_views_with_dropped_target_table", + materialized_view->getTargetTableId().getFullTableName(), + view_id.getFullTableName()); + } auto inner_table_id = inner_table->getStorageID(); auto inner_metadata_snapshot = inner_table->getInMemoryMetadataPtr(); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index f37abdfb1a3..8d7f3e5a9a7 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -71,6 +71,7 @@ public: StoragePtr getTargetTable() const; StoragePtr tryGetTargetTable() const; + const StorageID & getTargetTableId() const { return target_table_id; } /// Get the virtual column of the target table; NamesAndTypesList getVirtuals() const override; diff --git a/tests/queries/0_stateless/02479_race_condition_between_insert_and_droppin_mv.sh b/tests/queries/0_stateless/02479_race_condition_between_insert_and_droppin_mv.sh index 9ce4b459fce..6899b31d1d9 100755 --- a/tests/queries/0_stateless/02479_race_condition_between_insert_and_droppin_mv.sh +++ b/tests/queries/0_stateless/02479_race_condition_between_insert_and_droppin_mv.sh @@ -14,7 +14,7 @@ function insert { offset=500 while true; do - ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_race_condition_landing SELECT number, toString(number), toString(number) from system.numbers limit $i, $offset" + ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_race_condition_landing SELECT number, toString(number), toString(number) from system.numbers limit $i, $offset settings ignore_materialized_views_with_dropped_target_table=1" i=$(( $i + $RANDOM % 100 + 400 )) done } From 8a90f12dc9974c543d65a72653b565758c7e128c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 28 Dec 2023 23:28:26 +0100 Subject: [PATCH 08/61] Speedup MIN/MAX for non numeric types --- .../AggregateFunctionMax.cpp | 83 ++++++++++++++++++- .../AggregateFunctionMin.cpp | 83 ++++++++++++++++++- 2 files changed, 158 insertions(+), 8 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMax.cpp b/src/AggregateFunctions/AggregateFunctionMax.cpp index e74224a24c3..a440aedb62c 100644 --- a/src/AggregateFunctions/AggregateFunctionMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionMax.cpp @@ -19,7 +19,7 @@ public: explicit AggregateFunctionsSingleValueMax(const DataTypePtr & type) : Parent(type) { } /// Specializations for native numeric types - ALWAYS_INLINE inline void addBatchSinglePlace( + void addBatchSinglePlace( size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, @@ -27,7 +27,7 @@ public: Arena * arena, ssize_t if_argument_pos) const override; - ALWAYS_INLINE inline void addBatchSinglePlaceNotNull( + void addBatchSinglePlaceNotNull( size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, @@ -74,7 +74,50 @@ void AggregateFunctionsSingleValueMax::addBatchSinglePlace( Arena * arena, ssize_t if_argument_pos) const { - return Parent::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos); + constexpr int nan_direction_hint = 1; + auto const & column = *columns[0]; + if (if_argument_pos >= 0) + { + size_t index = row_begin; + const auto & if_flags = assert_cast(*columns[if_argument_pos]).getData(); + while (if_flags[index] == 0 && index < row_end) + index++; + if (index >= row_end) + return; + + for (size_t i = index + 1; i < row_end; i++) + { + if ((if_flags[i] != 0) && (column.compareAt(i, index, column, nan_direction_hint) > 0)) + index = i; + } + this->data(place).changeIfGreater(column, index, arena); + } + else + { + if (row_begin >= row_end) + return; + + /// TODO: Introduce row_begin and row_end to getPermutation + if (row_begin != 0 || row_end != column.size()) + { + size_t index = row_begin; + for (size_t i = index + 1; i < row_end; i++) + { + if (column.compareAt(i, index, column, nan_direction_hint) > 0) + index = i; + } + this->data(place).changeIfGreater(column, index, arena); + } + else + { + constexpr IColumn::PermutationSortDirection direction = IColumn::PermutationSortDirection::Descending; + constexpr IColumn::PermutationSortStability stability = IColumn::PermutationSortStability::Unstable; + IColumn::Permutation permutation; + constexpr UInt64 limit = 1; + column.getPermutation(direction, stability, limit, nan_direction_hint, permutation); + this->data(place).changeIfGreater(column, permutation[0], arena); + } + } } // NOLINTBEGIN(bugprone-macro-parentheses) @@ -119,7 +162,39 @@ void AggregateFunctionsSingleValueMax::addBatchSinglePlaceNotNull( Arena * arena, ssize_t if_argument_pos) const { - return Parent::addBatchSinglePlaceNotNull(row_begin, row_end, place, columns, null_map, arena, if_argument_pos); + constexpr int nan_direction_hint = 1; + auto const & column = *columns[0]; + if (if_argument_pos >= 0) + { + size_t index = row_begin; + const auto & if_flags = assert_cast(*columns[if_argument_pos]).getData(); + while ((if_flags[index] == 0 || null_map[index] != 0) && (index < row_end)) + index++; + if (index >= row_end) + return; + + for (size_t i = index + 1; i < row_end; i++) + { + if ((if_flags[i] != 0) && (null_map[i] == 0) && (column.compareAt(i, index, column, nan_direction_hint) > 0)) + index = i; + } + this->data(place).changeIfGreater(column, index, arena); + } + else + { + size_t index = row_begin; + while ((null_map[index] != 0) && (index < row_end)) + index++; + if (index >= row_end) + return; + + for (size_t i = index + 1; i < row_end; i++) + { + if ((null_map[i] == 0) && (column.compareAt(i, index, column, nan_direction_hint) > 0)) + index = i; + } + this->data(place).changeIfGreater(column, index, arena); + } } AggregateFunctionPtr createAggregateFunctionMax( diff --git a/src/AggregateFunctions/AggregateFunctionMin.cpp b/src/AggregateFunctions/AggregateFunctionMin.cpp index 48758aa74b0..8d5d12fa626 100644 --- a/src/AggregateFunctions/AggregateFunctionMin.cpp +++ b/src/AggregateFunctions/AggregateFunctionMin.cpp @@ -20,7 +20,7 @@ public: explicit AggregateFunctionsSingleValueMin(const DataTypePtr & type) : Parent(type) { } /// Specializations for native numeric types - ALWAYS_INLINE inline void addBatchSinglePlace( + void addBatchSinglePlace( size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, @@ -28,7 +28,7 @@ public: Arena * arena, ssize_t if_argument_pos) const override; - ALWAYS_INLINE inline void addBatchSinglePlaceNotNull( + void addBatchSinglePlaceNotNull( size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, @@ -75,7 +75,50 @@ void AggregateFunctionsSingleValueMin::addBatchSinglePlace( Arena * arena, ssize_t if_argument_pos) const { - return Parent::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos); + constexpr int nan_direction_hint = 1; + auto const & column = *columns[0]; + if (if_argument_pos >= 0) + { + size_t index = row_begin; + const auto & if_flags = assert_cast(*columns[if_argument_pos]).getData(); + while (if_flags[index] == 0 && index < row_end) + index++; + if (index >= row_end) + return; + + for (size_t i = index + 1; i < row_end; i++) + { + if ((if_flags[i] != 0) && (column.compareAt(i, index, column, nan_direction_hint) < 0)) + index = i; + } + this->data(place).changeIfLess(column, index, arena); + } + else + { + if (row_begin >= row_end) + return; + + /// TODO: Introduce row_begin and row_end to getPermutation + if (row_begin != 0 || row_end != column.size()) + { + size_t index = row_begin; + for (size_t i = index + 1; i < row_end; i++) + { + if (column.compareAt(i, index, column, nan_direction_hint) < 0) + index = i; + } + this->data(place).changeIfLess(column, index, arena); + } + else + { + constexpr IColumn::PermutationSortDirection direction = IColumn::PermutationSortDirection::Ascending; + constexpr IColumn::PermutationSortStability stability = IColumn::PermutationSortStability::Unstable; + IColumn::Permutation permutation; + constexpr UInt64 limit = 1; + column.getPermutation(direction, stability, limit, nan_direction_hint, permutation); + this->data(place).changeIfLess(column, permutation[0], arena); + } + } } // NOLINTBEGIN(bugprone-macro-parentheses) @@ -120,7 +163,39 @@ void AggregateFunctionsSingleValueMin::addBatchSinglePlaceNotNull( Arena * arena, ssize_t if_argument_pos) const { - return Parent::addBatchSinglePlaceNotNull(row_begin, row_end, place, columns, null_map, arena, if_argument_pos); + constexpr int nan_direction_hint = 1; + auto const & column = *columns[0]; + if (if_argument_pos >= 0) + { + size_t index = row_begin; + const auto & if_flags = assert_cast(*columns[if_argument_pos]).getData(); + while ((if_flags[index] == 0 || null_map[index] != 0) && (index < row_end)) + index++; + if (index >= row_end) + return; + + for (size_t i = index + 1; i < row_end; i++) + { + if ((if_flags[i] != 0) && (null_map[index] == 0) && (column.compareAt(i, index, column, nan_direction_hint) < 0)) + index = i; + } + this->data(place).changeIfLess(column, index, arena); + } + else + { + size_t index = row_begin; + while ((null_map[index] != 0) && (index < row_end)) + index++; + if (index >= row_end) + return; + + for (size_t i = index + 1; i < row_end; i++) + { + if ((null_map[i] == 0) && (column.compareAt(i, index, column, nan_direction_hint) < 0)) + index = i; + } + this->data(place).changeIfLess(column, index, arena); + } } AggregateFunctionPtr createAggregateFunctionMin( From e692b0a5bda00d14e109aeee1f1045b553183b10 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 29 Dec 2023 13:46:01 +0100 Subject: [PATCH 09/61] Move iota implementation to its own file --- src/Common/iota.cpp | 6 +++ src/Common/iota.h | 42 +++++++++++++++++++ .../QueryPlan/ReadFromSystemNumbersStep.cpp | 29 ++----------- 3 files changed, 52 insertions(+), 25 deletions(-) create mode 100644 src/Common/iota.cpp create mode 100644 src/Common/iota.h diff --git a/src/Common/iota.cpp b/src/Common/iota.cpp new file mode 100644 index 00000000000..7c0d28a66e0 --- /dev/null +++ b/src/Common/iota.cpp @@ -0,0 +1,6 @@ +#include + +namespace DB +{ +template void iota(UInt64 * begin, size_t count, UInt64 first_value); +} diff --git a/src/Common/iota.h b/src/Common/iota.h new file mode 100644 index 00000000000..d992032b77c --- /dev/null +++ b/src/Common/iota.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include +#include +#include + +/// This is a replacement for std::iota to use dynamic dispatch +/// Note that is only defined for containers with contiguous memory only + +namespace DB +{ + +/// Make sure to add any new type to the extern declaration at the end of the file and instantiate it in iota.cpp +template +concept iota_supported_types = (is_any_of); + +MULTITARGET_FUNCTION_AVX2_SSE42( + MULTITARGET_FUNCTION_HEADER(template void NO_INLINE), + iotaImpl, MULTITARGET_FUNCTION_BODY((T * begin, size_t count, T first_value) /// NOLINT + { + for (size_t i = 0; i < count; i++) + *(begin + i) = first_value + i; + }) +) + +template +void iota(T * begin, size_t count, T first_value) +{ +#if USE_MULTITARGET_CODE + if (isArchSupported(TargetArch::AVX2)) + return iotaImplAVX2(begin, count, first_value); + + if (isArchSupported(TargetArch::SSE42)) + return iotaImplSSE42(begin, count, first_value); +#endif + return iotaImpl(begin, count, first_value); +} + +extern template void iota(UInt64 * begin, size_t count, UInt64 first_value); + +} diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index a88203e0fca..329497d66d3 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include namespace DB @@ -23,27 +23,6 @@ extern const int TOO_MANY_ROWS; namespace { -MULTITARGET_FUNCTION_AVX2_SSE42( - MULTITARGET_FUNCTION_HEADER(void), - iotaImpl, MULTITARGET_FUNCTION_BODY((UInt64 * begin, UInt64 count, UInt64 first_value) - { - for (UInt64 i = 0; i < count; i++) - *(begin + i) = first_value + i; - }) -) - -static void iota(UInt64 * begin, UInt64 count, UInt64 first_value) -{ -#if USE_MULTITARGET_CODE - if (isArchSupported(TargetArch::AVX2)) - return iotaImplAVX2(begin, count, first_value); - - if (isArchSupported(TargetArch::SSE42)) - return iotaImplSSE42(begin, count, first_value); -#endif - return iotaImpl(begin, count, first_value); -} - class NumbersSource : public ISource { public: @@ -65,7 +44,7 @@ protected: size_t curr = next; /// The local variable for some reason works faster (>20%) than member of class. UInt64 * pos = vec.data(); /// This also accelerates the code. UInt64 * end = &vec[block_size]; - iota(pos, end - pos, curr); + iota(pos, static_cast(end - pos), curr); next += step; @@ -233,7 +212,7 @@ protected: auto start_value_64 = static_cast(start_value); auto end_value_64 = static_cast(end_value); auto size = end_value_64 - start_value_64; - iota(pos, size, start_value_64); + iota(pos, static_cast(size), start_value_64); pos += size; } }; @@ -242,7 +221,7 @@ protected: { UInt64 start_value = first_value(range) + cursor.offset_in_range; /// end_value will never overflow - iota(pos, need, start_value); + iota(pos, static_cast(need), start_value); pos += need; provided += need; From bda6104f84bdfce53115a728cd2e9d2f3251bc66 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 29 Dec 2023 14:38:22 +0100 Subject: [PATCH 10/61] Replace std::iota with DB::iota where possible --- ...ateFunctionLargestTriangleThreeBuckets.cpp | 5 ++-- src/AggregateFunctions/StatCommon.h | 3 +- src/Analyzer/Passes/FuseFunctionsPass.cpp | 3 +- src/Columns/ColumnObject.cpp | 3 +- src/Columns/tests/gtest_column_sparse.cpp | 3 +- src/Common/iota.cpp | 27 ++++++++++++++++++ src/Common/iota.h | 28 +++---------------- src/Common/tests/gtest_hash_table.cpp | 3 +- .../HashedDictionaryParallelLoader.h | 3 +- src/Dictionaries/PolygonDictionary.cpp | 3 +- src/Dictionaries/PolygonDictionaryUtils.h | 3 +- src/Functions/array/arrayRandomSample.cpp | 3 +- src/Functions/array/arrayShuffle.cpp | 3 +- src/Functions/translate.cpp | 5 ++-- src/Interpreters/tests/gtest_filecache.cpp | 3 +- 15 files changed, 59 insertions(+), 39 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp index 850a7c688ad..d5abdbc12fb 100644 --- a/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp +++ b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp @@ -14,8 +14,9 @@ #include #include #include -#include #include +#include +#include #include #include @@ -48,7 +49,7 @@ struct LargestTriangleThreeBucketsData : public StatisticalSamplex and this->y in ascending order of this->x using index std::vector index(this->x.size()); - std::iota(index.begin(), index.end(), 0); + iota(index.data(), index.size(), size_t(0)); ::sort(index.begin(), index.end(), [&](size_t i1, size_t i2) { return this->x[i1] < this->x[i2]; }); SampleX temp_x{}; diff --git a/src/AggregateFunctions/StatCommon.h b/src/AggregateFunctions/StatCommon.h index 23054e25189..8b1395ea95c 100644 --- a/src/AggregateFunctions/StatCommon.h +++ b/src/AggregateFunctions/StatCommon.h @@ -7,6 +7,7 @@ #include #include +#include #include #include @@ -30,7 +31,7 @@ std::pair computeRanksAndTieCorrection(const Values & value const size_t size = values.size(); /// Save initial positions, than sort indices according to the values. std::vector indexes(size); - std::iota(indexes.begin(), indexes.end(), 0); + iota(indexes.data(), indexes.size(), size_t(0)); std::sort(indexes.begin(), indexes.end(), [&] (size_t lhs, size_t rhs) { return values[lhs] < values[rhs]; }); diff --git a/src/Analyzer/Passes/FuseFunctionsPass.cpp b/src/Analyzer/Passes/FuseFunctionsPass.cpp index e77b3ddcb20..443e13b7d9d 100644 --- a/src/Analyzer/Passes/FuseFunctionsPass.cpp +++ b/src/Analyzer/Passes/FuseFunctionsPass.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -184,7 +185,7 @@ FunctionNodePtr createFusedQuantilesNode(std::vector & nodes { /// Sort nodes and parameters in ascending order of quantile level std::vector permutation(nodes.size()); - std::iota(permutation.begin(), permutation.end(), 0); + iota(permutation.data(), permutation.size(), size_t(0)); std::sort(permutation.begin(), permutation.end(), [&](size_t i, size_t j) { return parameters[i].get() < parameters[j].get(); }); std::vector new_nodes; diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 2052ec3c968..f7176568a1b 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -838,7 +839,7 @@ MutableColumnPtr ColumnObject::cloneResized(size_t new_size) const void ColumnObject::getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation & res) const { res.resize(num_rows); - std::iota(res.begin(), res.end(), 0); + iota(res.data(), res.size(), size_t(0)); } void ColumnObject::compareColumn(const IColumn & rhs, size_t rhs_row_num, diff --git a/src/Columns/tests/gtest_column_sparse.cpp b/src/Columns/tests/gtest_column_sparse.cpp index c3450ff91b4..02b15a2f5c4 100644 --- a/src/Columns/tests/gtest_column_sparse.cpp +++ b/src/Columns/tests/gtest_column_sparse.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -191,7 +192,7 @@ TEST(ColumnSparse, Permute) auto [sparse_src, full_src] = createColumns(n, k); IColumn::Permutation perm(n); - std::iota(perm.begin(), perm.end(), 0); + iota(perm.data(), perm.size(), size_t(0)); std::shuffle(perm.begin(), perm.end(), rng); auto sparse_dst = sparse_src->permute(perm, limit); diff --git a/src/Common/iota.cpp b/src/Common/iota.cpp index 7c0d28a66e0..385d3b22207 100644 --- a/src/Common/iota.cpp +++ b/src/Common/iota.cpp @@ -1,6 +1,33 @@ +#include #include +#include namespace DB { + +MULTITARGET_FUNCTION_AVX2_SSE42( + MULTITARGET_FUNCTION_HEADER(template void NO_INLINE), + iotaImpl, MULTITARGET_FUNCTION_BODY((T * begin, size_t count, T first_value) /// NOLINT + { + for (size_t i = 0; i < count; i++) + *(begin + i) = static_cast(first_value + i); + }) +) + +template +void iota(T * begin, size_t count, T first_value) +{ +#if USE_MULTITARGET_CODE + if (isArchSupported(TargetArch::AVX2)) + return iotaImplAVX2(begin, count, first_value); + + if (isArchSupported(TargetArch::SSE42)) + return iotaImplSSE42(begin, count, first_value); +#endif + return iotaImpl(begin, count, first_value); +} + +template void iota(UInt8 * begin, size_t count, UInt8 first_value); +template void iota(UInt32 * begin, size_t count, UInt32 first_value); template void iota(UInt64 * begin, size_t count, UInt64 first_value); } diff --git a/src/Common/iota.h b/src/Common/iota.h index d992032b77c..485df4bd4f0 100644 --- a/src/Common/iota.h +++ b/src/Common/iota.h @@ -1,9 +1,7 @@ #pragma once -#include #include #include -#include /// This is a replacement for std::iota to use dynamic dispatch /// Note that is only defined for containers with contiguous memory only @@ -13,30 +11,12 @@ namespace DB /// Make sure to add any new type to the extern declaration at the end of the file and instantiate it in iota.cpp template -concept iota_supported_types = (is_any_of); +concept iota_supported_types = (is_any_of); -MULTITARGET_FUNCTION_AVX2_SSE42( - MULTITARGET_FUNCTION_HEADER(template void NO_INLINE), - iotaImpl, MULTITARGET_FUNCTION_BODY((T * begin, size_t count, T first_value) /// NOLINT - { - for (size_t i = 0; i < count; i++) - *(begin + i) = first_value + i; - }) -) - -template -void iota(T * begin, size_t count, T first_value) -{ -#if USE_MULTITARGET_CODE - if (isArchSupported(TargetArch::AVX2)) - return iotaImplAVX2(begin, count, first_value); - - if (isArchSupported(TargetArch::SSE42)) - return iotaImplSSE42(begin, count, first_value); -#endif - return iotaImpl(begin, count, first_value); -} +template void iota(T * begin, size_t count, T first_value); +extern template void iota(UInt8 * begin, size_t count, UInt8 first_value); +extern template void iota(UInt32 * begin, size_t count, UInt32 first_value); extern template void iota(UInt64 * begin, size_t count, UInt64 first_value); } diff --git a/src/Common/tests/gtest_hash_table.cpp b/src/Common/tests/gtest_hash_table.cpp index 72941126cfd..ab7c3872170 100644 --- a/src/Common/tests/gtest_hash_table.cpp +++ b/src/Common/tests/gtest_hash_table.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -20,7 +21,7 @@ namespace std::vector getVectorWithNumbersUpToN(size_t n) { std::vector res(n); - std::iota(res.begin(), res.end(), 0); + iota(res.data(), res.size(), size_t(0)); return res; } diff --git a/src/Dictionaries/HashedDictionaryParallelLoader.h b/src/Dictionaries/HashedDictionaryParallelLoader.h index 907a987555e..ec892af7e36 100644 --- a/src/Dictionaries/HashedDictionaryParallelLoader.h +++ b/src/Dictionaries/HashedDictionaryParallelLoader.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -53,7 +54,7 @@ public: LOG_TRACE(dictionary.log, "Will load the dictionary using {} threads (with {} backlog)", shards, backlog); shards_slots.resize(shards); - std::iota(shards_slots.begin(), shards_slots.end(), 0); + iota(shards_slots.data(), shards_slots.size(), UInt64(0)); for (size_t shard = 0; shard < shards; ++shard) { diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index df3ae439b00..6f800bd921d 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -5,6 +5,7 @@ #include +#include #include #include #include @@ -507,7 +508,7 @@ const IColumn * unrollSimplePolygons(const ColumnPtr & column, Offset & offset) if (!ptr_polygons) throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column containing arrays of points"); offset.ring_offsets.assign(ptr_polygons->getOffsets()); - std::iota(offset.polygon_offsets.begin(), offset.polygon_offsets.end(), 1); + iota(offset.polygon_offsets.data(), offset.polygon_offsets.size(), IColumn::Offsets::value_type(1)); offset.multi_polygon_offsets.assign(offset.polygon_offsets); return ptr_polygons->getDataPtr().get(); diff --git a/src/Dictionaries/PolygonDictionaryUtils.h b/src/Dictionaries/PolygonDictionaryUtils.h index 0238ef0b2b9..63d97e9dabd 100644 --- a/src/Dictionaries/PolygonDictionaryUtils.h +++ b/src/Dictionaries/PolygonDictionaryUtils.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -184,7 +185,7 @@ public: { setBoundingBox(); std::vector order(polygons.size()); - std::iota(order.begin(), order.end(), 0); + iota(order.data(), order.size(), size_t(0)); root = makeCell(min_x, min_y, max_x, max_y, order); } diff --git a/src/Functions/array/arrayRandomSample.cpp b/src/Functions/array/arrayRandomSample.cpp index 1e28e089a2a..40344efb077 100644 --- a/src/Functions/array/arrayRandomSample.cpp +++ b/src/Functions/array/arrayRandomSample.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -80,7 +81,7 @@ public: const size_t cur_samples = std::min(num_elements, samples); indices.resize(num_elements); - std::iota(indices.begin(), indices.end(), prev_array_offset); + iota(indices.data(), indices.size(), prev_array_offset); std::shuffle(indices.begin(), indices.end(), rng); for (UInt64 i = 0; i < cur_samples; i++) diff --git a/src/Functions/array/arrayShuffle.cpp b/src/Functions/array/arrayShuffle.cpp index faa5ae47b29..10cb51d27d2 100644 --- a/src/Functions/array/arrayShuffle.cpp +++ b/src/Functions/array/arrayShuffle.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -150,7 +151,7 @@ ColumnPtr FunctionArrayShuffleImpl::executeGeneric(const ColumnArray & a size_t size = offsets.size(); size_t nested_size = array.getData().size(); IColumn::Permutation permutation(nested_size); - std::iota(std::begin(permutation), std::end(permutation), 0); + iota(permutation.data(), permutation.size(), IColumn::Permutation::value_type(0)); ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) diff --git a/src/Functions/translate.cpp b/src/Functions/translate.cpp index 836cb4de2f3..ad5be7d9dfd 100644 --- a/src/Functions/translate.cpp +++ b/src/Functions/translate.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -31,7 +32,7 @@ struct TranslateImpl if (map_from.size() != map_to.size()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second and third arguments must be the same length"); - std::iota(map.begin(), map.end(), 0); + iota(map.data(), map.size(), UInt8(0)); for (size_t i = 0; i < map_from.size(); ++i) { @@ -129,7 +130,7 @@ struct TranslateUTF8Impl if (map_from_size != map_to_size) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second and third arguments must be the same length"); - std::iota(map_ascii.begin(), map_ascii.end(), 0); + iota(map_ascii.data(), map_ascii.size(), UInt32(0)); const UInt8 * map_from_ptr = reinterpret_cast(map_from.data()); const UInt8 * map_from_end = map_from_ptr + map_from.size(); diff --git a/src/Interpreters/tests/gtest_filecache.cpp b/src/Interpreters/tests/gtest_filecache.cpp index 1005e6090b8..3e061db4f56 100644 --- a/src/Interpreters/tests/gtest_filecache.cpp +++ b/src/Interpreters/tests/gtest_filecache.cpp @@ -11,6 +11,7 @@ #include #include +#include #include #include #include @@ -788,7 +789,7 @@ TEST_F(FileCacheTest, writeBuffer) /// get random permutation of indexes std::vector indexes(data.size()); - std::iota(indexes.begin(), indexes.end(), 0); + iota(indexes.data(), indexes.size(), size_t(0)); std::shuffle(indexes.begin(), indexes.end(), rng); for (auto i : indexes) From bfc10bd234f2791fd48d30437e76df7d4a304a44 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 29 Dec 2023 15:16:12 +0100 Subject: [PATCH 11/61] an option to avoid waiting for inactive Replicated db replicas --- docs/en/operations/settings/settings.md | 2 + src/Core/SettingsEnums.cpp | 2 + src/Core/SettingsEnums.h | 2 + src/Interpreters/executeDDLQueryOnCluster.cpp | 98 ++++++++++++++----- .../test_replicated_database/test.py | 2 +- .../test.py | 2 +- ...distributed_ddl_output_mode_long.reference | 4 +- .../02447_drop_database_replica.reference | 8 ++ .../02447_drop_database_replica.sh | 3 + 9 files changed, 96 insertions(+), 27 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 6e087467bb9..d4ee8106320 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3847,6 +3847,8 @@ Possible values: - `none` — Is similar to throw, but distributed DDL query returns no result set. - `null_status_on_timeout` — Returns `NULL` as execution status in some rows of result set instead of throwing `TIMEOUT_EXCEEDED` if query is not finished on the corresponding hosts. - `never_throw` — Do not throw `TIMEOUT_EXCEEDED` and do not rethrow exceptions if query has failed on some hosts. +- `null_status_on_timeout_only_active` — similar to `null_status_on_timeout`, but doesn't wait for inactive replicas of the `Replicated` database +- `throw_only_active` — similar to `throw`, but doesn't wait for inactive replicas of the `Replicated` database Default value: `throw`. diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index c35e69977ed..2e6bb51176d 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -113,6 +113,8 @@ IMPLEMENT_SETTING_ENUM(DistributedDDLOutputMode, ErrorCodes::BAD_ARGUMENTS, {{"none", DistributedDDLOutputMode::NONE}, {"throw", DistributedDDLOutputMode::THROW}, {"null_status_on_timeout", DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT}, + {"throw_only_active", DistributedDDLOutputMode::THROW_ONLY_ACTIVE}, + {"null_status_on_timeout_only_active", DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE}, {"never_throw", DistributedDDLOutputMode::NEVER_THROW}}) IMPLEMENT_SETTING_ENUM(StreamingHandleErrorMode, ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 2e71c96b954..0b2d47210a8 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -165,6 +165,8 @@ enum class DistributedDDLOutputMode THROW, NULL_STATUS_ON_TIMEOUT, NEVER_THROW, + THROW_ONLY_ACTIVE, + NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE, }; DECLARE_SETTING_ENUM(DistributedDDLOutputMode) diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 9486350a0f6..ba7638cd83f 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -200,8 +200,6 @@ public: Status prepare() override; private: - static Strings getChildrenAllowNoNode(const std::shared_ptr & zookeeper, const String & node_path); - static Block getSampleBlock(ContextPtr context_, bool hosts_to_wait); Strings getNewAndUpdate(const Strings & current_list_of_finished_hosts); @@ -228,7 +226,8 @@ private: NameSet waiting_hosts; /// hosts from task host list NameSet finished_hosts; /// finished hosts from host list NameSet ignoring_hosts; /// appeared hosts that are not in hosts list - Strings current_active_hosts; /// Hosts that were in active state at the last check + Strings current_active_hosts; /// Hosts that are currently executing the task + NameSet offline_hosts; /// Hosts that are not currently running size_t num_hosts_finished = 0; /// Save the first detected error and throw it at the end of execution @@ -237,7 +236,10 @@ private: Int64 timeout_seconds = 120; bool is_replicated_database = false; bool throw_on_timeout = true; + bool only_running_hosts = false; + bool timeout_exceeded = false; + bool stop_waiting_offline_hosts = false; }; @@ -316,6 +318,8 @@ DDLQueryStatusSource::DDLQueryStatusSource( { waiting_hosts = NameSet(hosts_to_wait->begin(), hosts_to_wait->end()); is_replicated_database = true; + only_running_hosts = output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE || + output_mode == DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE; } else { @@ -377,6 +381,38 @@ Chunk DDLQueryStatusSource::generateChunkWithUnfinishedHosts() const return Chunk(std::move(columns), unfinished_hosts.size()); } +static NameSet getOfflineHosts(const String & node_path, const NameSet & hosts_to_wait, const ZooKeeperPtr & zookeeper, Poco::Logger * log) +{ + fs::path replicas_path; + if (node_path.ends_with('/')) + replicas_path = fs::path(node_path).parent_path().parent_path().parent_path() / "replicas"; + else + replicas_path = fs::path(node_path).parent_path().parent_path() / "replicas"; + + Strings paths; + Strings hosts_array; + for (const auto & host : hosts_to_wait) + { + hosts_array.push_back(host); + paths.push_back(replicas_path / host / "active"); + } + + NameSet offline; + auto res = zookeeper->tryGet(paths); + for (size_t i = 0; i < res.size(); ++i) + if (res[i].error == Coordination::Error::ZNONODE) + offline.insert(hosts_array[i]); + + if (offline.size() == hosts_to_wait.size()) + { + /// Avoid reporting that all hosts are offline + LOG_WARNING(log, "Did not find active hosts, will wait for all {} hosts. This should not happen often", offline.size()); + return {}; + } + + return offline; +} + Chunk DDLQueryStatusSource::generate() { bool all_hosts_finished = num_hosts_finished >= waiting_hosts.size(); @@ -398,7 +434,7 @@ Chunk DDLQueryStatusSource::generate() if (isCancelled()) return {}; - if (timeout_seconds >= 0 && watch.elapsedSeconds() > timeout_seconds) + if (stop_waiting_offline_hosts || (timeout_seconds >= 0 && watch.elapsedSeconds() > timeout_seconds)) { timeout_exceeded = true; @@ -406,7 +442,7 @@ Chunk DDLQueryStatusSource::generate() size_t num_active_hosts = current_active_hosts.size(); constexpr auto msg_format = "Watching task {} is executing longer than distributed_ddl_task_timeout (={}) seconds. " - "There are {} unfinished hosts ({} of them are currently active), " + "There are {} unfinished hosts ({} of them are currently executing the task), " "they are going to execute the query in background"; if (throw_on_timeout) { @@ -425,10 +461,7 @@ Chunk DDLQueryStatusSource::generate() return generateChunkWithUnfinishedHosts(); } - if (num_hosts_finished != 0 || try_number != 0) - { - sleepForMilliseconds(std::min(1000, 50 * (try_number + 1))); - } + sleepForMilliseconds(std::min(1000, 50 * try_number)); bool node_exists = false; Strings tmp_hosts; @@ -440,9 +473,21 @@ Chunk DDLQueryStatusSource::generate() retries_ctl.retryLoop([&]() { auto zookeeper = context->getZooKeeper(); - node_exists = zookeeper->exists(node_path); - tmp_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / node_to_wait); - tmp_active_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / "active"); + Strings paths = {String(fs::path(node_path) / node_to_wait), String(fs::path(node_path) / "active")}; + auto res = zookeeper->tryGetChildren(paths); + for (size_t i = 0; i < res.size(); ++i) + if (res[i].error != Coordination::Error::ZOK && res[i].error != Coordination::Error::ZNONODE) + throw Coordination::Exception::fromPath(res[i].error, paths[i]); + + if (res[0].error == Coordination::Error::ZNONODE) + node_exists = zookeeper->exists(node_path); + else + node_exists = true; + tmp_hosts = res[0].names; + tmp_active_hosts = res[1].names; + + if (only_running_hosts) + offline_hosts = getOfflineHosts(node_path, waiting_hosts, zookeeper, log); }); } @@ -460,6 +505,17 @@ Chunk DDLQueryStatusSource::generate() Strings new_hosts = getNewAndUpdate(tmp_hosts); ++try_number; + + if (only_running_hosts) + { + size_t num_finished_or_offline = 0; + for (const auto & host : waiting_hosts) + num_finished_or_offline += finished_hosts.contains(host) || offline_hosts.contains(host); + + if (num_finished_or_offline == waiting_hosts.size()) + stop_waiting_offline_hosts = true; + } + if (new_hosts.empty()) continue; @@ -470,7 +526,13 @@ Chunk DDLQueryStatusSource::generate() { ExecutionStatus status(-1, "Cannot obtain error message"); - if (node_to_wait == "finished") + /// Replicated database retries in case of error, it should not write error status. +#ifdef ABORT_ON_LOGICAL_ERROR + bool need_check_status = true; +#else + bool need_check_status = !is_replicated_database; +#endif + if (need_check_status) { String status_data; bool finished_exists = false; @@ -496,7 +558,6 @@ Chunk DDLQueryStatusSource::generate() if (status.code != 0 && !first_exception && context->getSettingsRef().distributed_ddl_output_mode != DistributedDDLOutputMode::NEVER_THROW) { - /// Replicated database retries in case of error, it should not write error status. if (is_replicated_database) throw Exception(ErrorCodes::LOGICAL_ERROR, "There was an error on {}: {} (probably it's a bug)", host_id, status.message); @@ -555,15 +616,6 @@ IProcessor::Status DDLQueryStatusSource::prepare() return ISource::prepare(); } -Strings DDLQueryStatusSource::getChildrenAllowNoNode(const std::shared_ptr & zookeeper, const String & node_path) -{ - Strings res; - Coordination::Error code = zookeeper->tryGetChildren(node_path, res); - if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) - throw Coordination::Exception::fromPath(code, node_path); - return res; -} - Strings DDLQueryStatusSource::getNewAndUpdate(const Strings & current_list_of_finished_hosts) { Strings diff; diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 3ced82ebb57..1fc3fe37044 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -507,7 +507,7 @@ def test_alters_from_different_replicas(started_cluster): settings = {"distributed_ddl_task_timeout": 5} assert ( - "There are 1 unfinished hosts (0 of them are currently active)" + "There are 1 unfinished hosts (0 of them are currently executing the task" in competing_node.query_and_get_error( "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN Added0 UInt32;", settings=settings, diff --git a/tests/integration/test_replicated_database_cluster_groups/test.py b/tests/integration/test_replicated_database_cluster_groups/test.py index b14581c1fe6..647626d8014 100644 --- a/tests/integration/test_replicated_database_cluster_groups/test.py +++ b/tests/integration/test_replicated_database_cluster_groups/test.py @@ -96,7 +96,7 @@ def test_cluster_groups(started_cluster): main_node_2.stop_clickhouse() settings = {"distributed_ddl_task_timeout": 5} assert ( - "There are 1 unfinished hosts (0 of them are currently active)" + "There are 1 unfinished hosts (0 of them are currently executing the task)" in main_node_1.query_and_get_error( "CREATE TABLE cluster_groups.table_2 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);", settings=settings, diff --git a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference index 39979a98bde..b9a66a1e1a9 100644 --- a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference +++ b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference @@ -3,7 +3,7 @@ Received exception from server: Code: 57. Error: Received from localhost:9000. Error: There was an error on [localhost:9000]: Code: 57. Error: Table default.none already exists. (TABLE_ALREADY_EXISTS) (query: create table none on cluster test_shard_localhost (n int) engine=Memory;) Received exception from server: -Code: 159. Error: Received from localhost:9000. Error: Watching task is executing longer than distributed_ddl_task_timeout (=1) seconds. There are 1 unfinished hosts (0 of them are currently active), they are going to execute the query in background. (TIMEOUT_EXCEEDED) +Code: 159. Error: Received from localhost:9000. Error: Watching task is executing longer than distributed_ddl_task_timeout (=1) seconds. There are 1 unfinished hosts (0 of them are currently executing the task), they are going to execute the query in background. (TIMEOUT_EXCEEDED) (query: drop table if exists none on cluster test_unavailable_shard;) throw localhost 9000 0 0 0 @@ -12,7 +12,7 @@ Code: 57. Error: Received from localhost:9000. Error: There was an error on [loc (query: create table throw on cluster test_shard_localhost (n int) engine=Memory format Null;) localhost 9000 0 1 0 Received exception from server: -Code: 159. Error: Received from localhost:9000. Error: Watching task is executing longer than distributed_ddl_task_timeout (=1) seconds. There are 1 unfinished hosts (0 of them are currently active), they are going to execute the query in background. (TIMEOUT_EXCEEDED) +Code: 159. Error: Received from localhost:9000. Error: Watching task is executing longer than distributed_ddl_task_timeout (=1) seconds. There are 1 unfinished hosts (0 of them are currently executing the task), they are going to execute the query in background. (TIMEOUT_EXCEEDED) (query: drop table if exists throw on cluster test_unavailable_shard;) null_status_on_timeout localhost 9000 0 0 0 diff --git a/tests/queries/0_stateless/02447_drop_database_replica.reference b/tests/queries/0_stateless/02447_drop_database_replica.reference index f2b41569540..8ad9008057f 100644 --- a/tests/queries/0_stateless/02447_drop_database_replica.reference +++ b/tests/queries/0_stateless/02447_drop_database_replica.reference @@ -12,10 +12,18 @@ t 2 rdb_default 1 1 s1 r1 1 2 +s1 r1 OK 2 0 +s2 r1 QUEUED 2 0 +s1 r2 QUEUED 2 0 +s1 r1 OK 2 0 +s2 r1 QUEUED 2 0 +s1 r2 QUEUED 2 0 2 rdb_default 1 1 s1 r1 1 rdb_default 1 2 s1 r2 0 2 2 t +t2 +t3 rdb_default_4 1 1 s1 r1 1 diff --git a/tests/queries/0_stateless/02447_drop_database_replica.sh b/tests/queries/0_stateless/02447_drop_database_replica.sh index d5b3ceef46a..388af3fad74 100755 --- a/tests/queries/0_stateless/02447_drop_database_replica.sh +++ b/tests/queries/0_stateless/02447_drop_database_replica.sh @@ -32,6 +32,9 @@ $CLICKHOUSE_CLIENT -q "system sync database replica $db" $CLICKHOUSE_CLIENT -q "select cluster, shard_num, replica_num, database_shard_name, database_replica_name, is_active from system.clusters where cluster='$db' and shard_num=1 and replica_num=1" $CLICKHOUSE_CLIENT -q "system drop database replica 's1|r1' from database $db2" 2>&1| grep -Fac "is active, cannot drop it" +$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=throw_only_active -q "create table $db.t2 (n int) engine=Log" +$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=null_status_on_timeout_only_active -q "create table $db.t3 (n int) engine=Log" + $CLICKHOUSE_CLIENT -q "detach database $db3" $CLICKHOUSE_CLIENT -q "system drop database replica 'r1' from shard 's2' from database $db" $CLICKHOUSE_CLIENT -q "attach database $db3" 2>/dev/null From 5521e5d9b16a7527d81cf97742c548570769d143 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 29 Dec 2023 15:58:01 +0000 Subject: [PATCH 12/61] Refactor StorageHDFS and StorageFile virtual columns filtering --- src/Storages/HDFS/StorageHDFS.cpp | 185 ++++++++++++++++++++++++---- src/Storages/HDFS/StorageHDFS.h | 12 +- src/Storages/StorageFile.cpp | 158 ++++++++++++++++++++---- src/Storages/StorageFile.h | 16 ++- src/Storages/VirtualColumnUtils.cpp | 36 ++++++ src/Storages/VirtualColumnUtils.h | 19 +++ 6 files changed, 369 insertions(+), 57 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index fdbb5e9f171..9d719413c8d 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -15,6 +15,8 @@ #include #include #include +#include +#include #include #include @@ -408,6 +410,35 @@ ColumnsDescription StorageHDFS::getTableStructureFromData( class HDFSSource::DisclosedGlobIterator::Impl { public: + + Impl(const String & uri, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) + { + const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); + uris = getPathsList(path_from_uri, uri_without_path, context); + ActionsDAGPtr filter_dag; + if (!uris.empty()) + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, uris[0].path); + + if (filter_dag) + { + std::vector paths; + paths.reserve(uris.size()); + for (const auto & path_with_info : uris) + paths.push_back(path_with_info.path); + + VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, context); + } + auto file_progress_callback = context->getFileProgressCallback(); + + for (auto & elem : uris) + { + elem.path = uri_without_path + elem.path; + if (file_progress_callback && elem.info) + file_progress_callback(FileProgress(0, elem.info->size)); + } + uris_iter = uris.begin(); + } + Impl(const String & uri, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context) { const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); @@ -456,21 +487,21 @@ private: class HDFSSource::URISIterator::Impl : WithContext { public: - explicit Impl(const std::vector & uris_, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context_) + explicit Impl(const std::vector & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context_) : WithContext(context_), uris(uris_), file_progress_callback(context_->getFileProgressCallback()) { - ASTPtr filter_ast; + ActionsDAGPtr filter_dag; if (!uris.empty()) - filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, getPathFromUriAndUriWithoutPath(uris[0]).first, getContext()); + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, getPathFromUriAndUriWithoutPath(uris[0]).first); - if (filter_ast) + if (filter_dag) { std::vector paths; paths.reserve(uris.size()); for (const auto & uri : uris) paths.push_back(getPathFromUriAndUriWithoutPath(uri).first); - VirtualColumnUtils::filterByPathOrFile(uris, paths, query, virtual_columns, getContext(), filter_ast); + VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, getContext()); } if (!uris.empty()) @@ -520,13 +551,16 @@ private: HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context) : pimpl(std::make_shared(uri, query, virtual_columns, context)) {} +HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) + : pimpl(std::make_shared(uri, predicate, virtual_columns, context)) {} + StorageHDFS::PathWithInfo HDFSSource::DisclosedGlobIterator::next() { return pimpl->next(); } -HDFSSource::URISIterator::URISIterator(const std::vector & uris_, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - : pimpl(std::make_shared(uris_, query, virtual_columns, context)) +HDFSSource::URISIterator::URISIterator(const std::vector & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) + : pimpl(std::make_shared(uris_, predicate, virtual_columns, context)) { } @@ -541,8 +575,8 @@ HDFSSource::HDFSSource( ContextPtr context_, UInt64 max_block_size_, std::shared_ptr file_iterator_, - bool need_only_count_, - const SelectQueryInfo & query_info_) + bool need_only_count_) + //const SelectQueryInfo & query_info_) : ISource(info.source_header, false) , WithContext(context_) , storage(std::move(storage_)) @@ -553,7 +587,7 @@ HDFSSource::HDFSSource( , file_iterator(file_iterator_) , columns_description(info.columns_description) , need_only_count(need_only_count_) - , query_info(query_info_) + //, query_info(query_info_) { initialize(); } @@ -843,7 +877,82 @@ bool StorageHDFS::supportsSubsetOfColumns(const ContextPtr & context_) const return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name, context_); } -Pipe StorageHDFS::read( +class ReadFromHDFS : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromHDFS"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + void applyFilters() override; + + ReadFromHDFS( + Block sample_block, + std::vector uris_, + bool distributed_processing_, + NamesAndTypesList virtual_columns_, + bool is_path_with_globs_, + ReadFromFormatInfo info_, + bool need_only_count_, + std::shared_ptr storage_, + // StorageSnapshotPtr storage_snapshot_, + // const StorageEmbeddedRocksDB & storage_, + // SelectQueryInfo query_info_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_) + : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) + , uris(std::move(uris_)) + , distributed_processing(distributed_processing_) + , virtual_columns(std::move(virtual_columns_)) + , is_path_with_globs(is_path_with_globs_) + , info(std::move(info_)) + , need_only_count(need_only_count_) + , storage(std::move(storage_)) + // , storage_snapshot(std::move(storage_snapshot_)) + // , storage(storage_) + // , query_info(std::move(query_info_)) + , context(std::move(context_)) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + { + } + +private: + std::vector uris; + const bool distributed_processing; + NamesAndTypesList virtual_columns; + bool is_path_with_globs; + ReadFromFormatInfo info; + const bool need_only_count; + std::shared_ptr storage; + + // StorageSnapshotPtr storage_snapshot; + // const StorageEmbeddedRocksDB & storage; + // SelectQueryInfo query_info; + ContextPtr context; + + size_t max_block_size; + size_t num_streams; + + std::shared_ptr iterator_wrapper; + + // FieldVectorPtr keys; + // bool all_scan = false; + + void createIterator(const ActionsDAG::Node * predicate); +}; + +void ReadFromHDFS::applyFilters() +{ + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + createIterator(predicate); +} + +void StorageHDFS::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -852,18 +961,44 @@ Pipe StorageHDFS::read( size_t max_block_size, size_t num_streams) { - std::shared_ptr iterator_wrapper{nullptr}; + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context_), virtual_columns); + bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) + && context_->getSettingsRef().optimize_count_from_files; + + auto this_ptr = std::static_pointer_cast(shared_from_this()); + + auto reading = std::make_unique( + read_from_format_info.source_header, + uris, + distributed_processing, + virtual_columns, + is_path_with_globs, + std::move(read_from_format_info), + need_only_count, + std::move(this_ptr), + context_, + max_block_size, + num_streams); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromHDFS::createIterator(const ActionsDAG::Node * predicate) +{ + if (iterator_wrapper) + return; + if (distributed_processing) { iterator_wrapper = std::make_shared( - [callback = context_->getReadTaskCallback()]() -> StorageHDFS::PathWithInfo { + [callback = context->getReadTaskCallback()]() -> StorageHDFS::PathWithInfo { return StorageHDFS::PathWithInfo{callback(), std::nullopt}; }); } else if (is_path_with_globs) { /// Iterate through disclosed globs and make a source for each file - auto glob_iterator = std::make_shared(uris[0], query_info.query, virtual_columns, context_); + auto glob_iterator = std::make_shared(uris[0], predicate, virtual_columns, context); iterator_wrapper = std::make_shared([glob_iterator]() { return glob_iterator->next(); @@ -871,31 +1006,31 @@ Pipe StorageHDFS::read( } else { - auto uris_iterator = std::make_shared(uris, query_info.query, virtual_columns, context_); + auto uris_iterator = std::make_shared(uris, predicate, virtual_columns, context); iterator_wrapper = std::make_shared([uris_iterator]() { return uris_iterator->next(); }); } +} - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context_), getVirtuals()); - bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) - && context_->getSettingsRef().optimize_count_from_files; +void ReadFromHDFS::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + createIterator(nullptr); Pipes pipes; - auto this_ptr = std::static_pointer_cast(shared_from_this()); for (size_t i = 0; i < num_streams; ++i) { pipes.emplace_back(std::make_shared( - read_from_format_info, - this_ptr, - context_, + info, + storage, + context, max_block_size, iterator_wrapper, - need_only_count, - query_info)); + need_only_count)); //, + //query_info)); } - return Pipe::unitePipes(std::move(pipes)); + pipeline.init(Pipe::unitePipes(std::move(pipes))); } SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context_, bool /*async_insert*/) diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 18eeb787d77..cee1b674eb7 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -51,7 +51,8 @@ public: String getName() const override { return "HDFS"; } - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -115,6 +116,7 @@ public: { public: DisclosedGlobIterator(const String & uri_, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context); + DisclosedGlobIterator(const String & uri_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context); StorageHDFS::PathWithInfo next(); private: class Impl; @@ -125,7 +127,7 @@ public: class URISIterator { public: - URISIterator(const std::vector & uris_, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context); + URISIterator(const std::vector & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context); StorageHDFS::PathWithInfo next(); private: class Impl; @@ -142,8 +144,8 @@ public: ContextPtr context_, UInt64 max_block_size_, std::shared_ptr file_iterator_, - bool need_only_count_, - const SelectQueryInfo & query_info_); + bool need_only_count_); + //const SelectQueryInfo & query_info_); String getName() const override; @@ -162,7 +164,7 @@ private: ColumnsDescription columns_description; bool need_only_count; size_t total_rows_in_file = 0; - SelectQueryInfo query_info; + //SelectQueryInfo query_info; std::unique_ptr read_buf; std::shared_ptr input_format; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 25bb6691ff6..b040f452410 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -44,6 +45,8 @@ #include #include #include +#include +#include #include #include @@ -947,6 +950,23 @@ StorageFileSource::FilesIterator::FilesIterator( VirtualColumnUtils::filterByPathOrFile(files, files, query, virtual_columns, context_, filter_ast); } +StorageFileSource::FilesIterator::FilesIterator( + const Strings & files_, + std::optional archive_info_, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns, + ContextPtr context_, + bool distributed_processing_) + : files(files_), archive_info(std::move(archive_info_)), distributed_processing(distributed_processing_), context(context_) +{ + ActionsDAGPtr filter_dag; + if (!distributed_processing && !archive_info && !files.empty() && !files[0].empty()) + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, files[0]); + + if (filter_dag) + VirtualColumnUtils::filterByPathOrFile(files, files, filter_dag, virtual_columns, context_); +} + String StorageFileSource::FilesIterator::next() { if (distributed_processing) @@ -974,16 +994,13 @@ const String & StorageFileSource::FilesIterator::getFileNameInArchive() StorageFileSource::StorageFileSource( const ReadFromFormatInfo & info, std::shared_ptr storage_, - const StorageSnapshotPtr & storage_snapshot_, ContextPtr context_, - const SelectQueryInfo & query_info_, UInt64 max_block_size_, FilesIteratorPtr files_iterator_, std::unique_ptr read_buf_, bool need_only_count_) : SourceWithKeyCondition(info.source_header, false) , storage(std::move(storage_)) - , storage_snapshot(storage_snapshot_) , files_iterator(std::move(files_iterator_)) , read_buf(std::move(read_buf_)) , columns_description(info.columns_description) @@ -991,7 +1008,6 @@ StorageFileSource::StorageFileSource( , requested_virtual_columns(info.requested_virtual_columns) , block_for_format(info.format_header) , context(context_) - , query_info(query_info_) , max_block_size(max_block_size_) , need_only_count(need_only_count_) { @@ -1322,14 +1338,87 @@ std::optional StorageFileSource::tryGetNumRowsFromCache(const String & p return schema_cache.tryGetNumRows(key, get_last_mod_time); } -Pipe StorageFile::read( +class ReadFromFile : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromFile"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + void applyFilters() override; + + ReadFromFile( + Block sample_block, + std::shared_ptr storage_, + std::vector paths_, + std::optional archive_info_, + NamesAndTypesList virtual_columns_, + bool distributed_processing_, + ReadFromFormatInfo info_, + const bool need_only_count_, + size_t total_bytes_to_read_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_) + : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) + , storage(std::move(storage_)) + , paths(std::move(paths_)) + , archive_info(std::move(archive_info_)) + , virtual_columns(std::move(virtual_columns_)) + , distributed_processing(distributed_processing_) + , info(std::move(info_)) + , need_only_count(need_only_count_) + , total_bytes_to_read(total_bytes_to_read_) + , context(std::move(context_)) + , max_block_size(max_block_size_) + , max_num_streams(num_streams_) + { + } + +private: + std::shared_ptr storage; + + std::vector paths; + std::optional archive_info; + + NamesAndTypesList virtual_columns; + const bool distributed_processing; + + ReadFromFormatInfo info; + const bool need_only_count; + + size_t total_bytes_to_read; + + ContextPtr context; + + size_t max_block_size; + const size_t max_num_streams; + + std::shared_ptr files_iterator; + + // FieldVectorPtr keys; + // bool all_scan = false; + + void createIterator(const ActionsDAG::Node * predicate); +}; + +void ReadFromFile::applyFilters() +{ + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + createIterator(predicate); +} + +void StorageFile::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - const size_t max_num_streams) + size_t num_streams) { if (use_table_fd) { @@ -1346,17 +1435,48 @@ Pipe StorageFile::read( if (p->size() == 1 && !fs::exists(p->at(0))) { - if (context->getSettingsRef().engine_file_empty_if_not_exists) - return Pipe(std::make_shared(storage_snapshot->getSampleBlockForColumns(column_names))); - else + if (!context->getSettingsRef().engine_file_empty_if_not_exists) throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File {} doesn't exist", p->at(0)); + + auto header = storage_snapshot->getSampleBlockForColumns(column_names); + InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, context); + return; } } - auto files_iterator = std::make_shared(paths, archive_info, query_info.query, virtual_columns, context, distributed_processing); - auto this_ptr = std::static_pointer_cast(shared_from_this()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context), getVirtuals()); + bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) + && context->getSettingsRef().optimize_count_from_files; + + auto reading = std::make_unique( + read_from_format_info.source_header, + std::move(this_ptr), + paths, + archive_info, + virtual_columns, + distributed_processing, + std::move(read_from_format_info), + need_only_count, + total_bytes_to_read, + context, + max_block_size, + num_streams); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromFile::createIterator(const ActionsDAG::Node * predicate) +{ + if (files_iterator) + return; + + files_iterator = std::make_shared(paths, archive_info, predicate, virtual_columns, context, distributed_processing); +} + +void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ size_t num_streams = max_num_streams; size_t files_to_read = 0; @@ -1377,10 +1497,6 @@ Pipe StorageFile::read( if (progress_callback && !archive_info) progress_callback(FileProgress(0, total_bytes_to_read)); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context), getVirtuals()); - bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) - && context->getSettingsRef().optimize_count_from_files; - for (size_t i = 0; i < num_streams; ++i) { /// In case of reading from fd we have to check whether we have already created @@ -1388,22 +1504,20 @@ Pipe StorageFile::read( /// If yes, then we should use it in StorageFileSource. Atomic bool flag is needed /// to prevent data race in case of parallel reads. std::unique_ptr read_buffer; - if (has_peekable_read_buffer_from_fd.exchange(false)) - read_buffer = std::move(peekable_read_buffer_from_fd); + if (storage->has_peekable_read_buffer_from_fd.exchange(false)) + read_buffer = std::move(storage->peekable_read_buffer_from_fd); pipes.emplace_back(std::make_shared( - read_from_format_info, - this_ptr, - storage_snapshot, + info, + storage, context, - query_info, max_block_size, files_iterator, std::move(read_buffer), need_only_count)); } - return Pipe::unitePipes(std::move(pipes)); + pipeline.init(Pipe::unitePipes(std::move(pipes))); } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 1fd3f2e0edf..ecb9e01b862 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -53,7 +53,8 @@ public: std::string getName() const override { return "File"; } - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -137,6 +138,7 @@ public: protected: friend class StorageFileSource; friend class StorageFileSink; + friend class ReadFromFile; private: void setStorageMetadata(CommonArguments args); @@ -199,6 +201,14 @@ public: ContextPtr context_, bool distributed_processing_ = false); + explicit FilesIterator( + const Strings & files_, + std::optional archive_info_, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns, + ContextPtr context_, + bool distributed_processing_ = false); + String next(); bool isReadFromArchive() const @@ -234,9 +244,7 @@ private: StorageFileSource( const ReadFromFormatInfo & info, std::shared_ptr storage_, - const StorageSnapshotPtr & storage_snapshot_, ContextPtr context_, - const SelectQueryInfo & query_info_, UInt64 max_block_size_, FilesIteratorPtr files_iterator_, std::unique_ptr read_buf_, @@ -269,7 +277,6 @@ private: std::optional tryGetNumRowsFromCache(const String & path, time_t last_mod_time) const; std::shared_ptr storage; - StorageSnapshotPtr storage_snapshot; FilesIteratorPtr files_iterator; String current_path; std::optional current_file_size; @@ -290,7 +297,6 @@ private: Block block_for_format; ContextPtr context; /// TODO Untangle potential issues with context lifetime. - SelectQueryInfo query_info; UInt64 max_block_size; bool finished_generate = false; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index aed06fb0540..7690e160255 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -390,6 +390,42 @@ static void addPathAndFileToVirtualColumns(Block & block, const String & path, s block.getByName("_idx").column->assumeMutableRef().insert(idx); } +ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const String & path_example) +{ + if (!predicate || virtual_columns.empty()) + return {}; + + Block block; + for (const auto & column : virtual_columns) + { + if (column.name == "_file" || column.name == "_path") + block.insert({column.type->createColumn(), column.type, column.name}); + } + /// Create a block with one row to construct filter + /// Append "idx" column as the filter result + block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); + addPathAndFileToVirtualColumns(block, path_example, 0); + return splitFilterDagForAllowedInputs(predicate, block); +} + +ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context) +{ + Block block; + for (const auto & column : virtual_columns) + { + if (column.name == "_file" || column.name == "_path") + block.insert({column.type->createColumn(), column.type, column.name}); + } + block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); + + for (size_t i = 0; i != paths.size(); ++i) + addPathAndFileToVirtualColumns(block, paths[i], i); + + filterBlockWithDAG(dag, block, context); + + return block.getByName("_idx").column; +} + ASTPtr createPathAndFileFilterAst(const ASTPtr & query, const NamesAndTypesList & virtual_columns, const String & path_example, const ContextPtr & context) { if (!query || virtual_columns.empty()) diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index e22b9742888..4f9636b4213 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -77,6 +77,25 @@ void filterByPathOrFile(std::vector & sources, const std::vector & pa sources = std::move(filtered_sources); } +ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const String & path_example); + +ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context); + +template +void filterByPathOrFile(std::vector & sources, const std::vector & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context) +{ + auto indexes_column = getFilterByPathAndFileIndexes(paths, dag, virtual_columns, context); + const auto & indexes = typeid_cast(*indexes_column).getData(); + if (indexes.size() == sources.size()) + return; + + std::vector filtered_sources; + filtered_sources.reserve(indexes.size()); + for (auto index : indexes) + filtered_sources.emplace_back(std::move(sources[index])); + sources = std::move(filtered_sources); +} + void addRequestedPathFileAndSizeVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename = nullptr); } From db97a6998901aeb0a60f2a9cb57bcb98a75881e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 29 Dec 2023 17:00:01 +0100 Subject: [PATCH 13/61] Add perf tests with tuples --- tests/performance/agg_functions_min_max_any.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/performance/agg_functions_min_max_any.xml b/tests/performance/agg_functions_min_max_any.xml index 2926a5ed3c8..f8469244643 100644 --- a/tests/performance/agg_functions_min_max_any.xml +++ b/tests/performance/agg_functions_min_max_any.xml @@ -87,4 +87,9 @@ select any(FromTag) from hits_100m_single where FromTag != '' group by intHash32(UserID) % {group_scale} FORMAT Null select anyHeavy(FromTag) from hits_100m_single where FromTag != '' group by intHash32(UserID) % {group_scale} FORMAT Null + +select min((WatchID, CounterID)) from hits_100m_single FORMAT Null +select max((WatchID, CounterID)) from hits_100m_single FORMAT Null +select any((WatchID, CounterID)) from hits_100m_single FORMAT Null +select anyHeavy((WatchID, CounterID)) from hits_100m_single FORMAT Null From a38b3b9a7945fcca64f42a230ac9df808790a70a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 29 Dec 2023 17:02:02 +0100 Subject: [PATCH 14/61] Fix test --- ..._materialized_view_with_dropped_target_table_no_exception.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.sql b/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.sql index 744b2578617..af6dbf24473 100644 --- a/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.sql +++ b/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.sql @@ -1,3 +1,4 @@ +set ignore_materialized_views_with_dropped_target_table = 1; drop table if exists from_table; drop table if exists to_table; drop table if exists mv; From 4b7fcfbc75d5ffe5d4331f2370d43537e504bc44 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 29 Dec 2023 14:56:08 +0100 Subject: [PATCH 15/61] Use iota in more places --- src/AggregateFunctions/QuantilesCommon.h | 4 ++-- src/Columns/IColumnImpl.h | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/AggregateFunctions/QuantilesCommon.h b/src/AggregateFunctions/QuantilesCommon.h index 3dda0119485..afbca84b827 100644 --- a/src/AggregateFunctions/QuantilesCommon.h +++ b/src/AggregateFunctions/QuantilesCommon.h @@ -6,6 +6,7 @@ #include #include +#include namespace DB @@ -63,10 +64,9 @@ struct QuantileLevels if (isNaN(levels[i]) || levels[i] < 0 || levels[i] > 1) throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Quantile level is out of range [0..1]"); - - permutation[i] = i; } + iota(permutation.data(), size, Permutation::value_type(0)); ::sort(permutation.begin(), permutation.end(), [this] (size_t a, size_t b) { return levels[a] < levels[b]; }); } }; diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index 0eab9452813..8e0bf0014f2 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -6,10 +6,11 @@ * implementation. */ -#include -#include -#include #include +#include +#include +#include +#include namespace DB @@ -299,8 +300,7 @@ void IColumn::getPermutationImpl( if (limit >= data_size) limit = 0; - for (size_t i = 0; i < data_size; ++i) - res[i] = i; + iota(res.data(), data_size, Permutation::value_type(0)); if (limit) { From ed6b9703a1a4848949f6e6f37a241a0cffb17c96 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 29 Dec 2023 15:57:36 +0100 Subject: [PATCH 16/61] More iota --- src/Columns/ColumnAggregateFunction.cpp | 18 +++++++++--------- src/Columns/ColumnConst.cpp | 8 ++++---- src/Columns/ColumnDecimal.cpp | 15 +++++++-------- src/Columns/ColumnSparse.cpp | 14 +++++++------- src/Columns/ColumnTuple.cpp | 12 ++++++------ src/Columns/ColumnVector.cpp | 18 +++++++++--------- src/Columns/IColumnDummy.cpp | 10 +++++----- .../tests/gtest_column_stable_permutation.cpp | 10 +++------- src/Common/levenshteinDistance.cpp | 6 +++--- src/Functions/FunctionsStringDistance.cpp | 4 ++-- src/Functions/array/arraySort.cpp | 7 +++---- src/Functions/rowNumberInBlock.cpp | 3 +-- src/Interpreters/sortBlock.cpp | 4 ++-- .../Transforms/PartialSortingTransform.cpp | 9 ++++----- src/QueryPipeline/QueryPipelineBuilder.cpp | 14 +++++++------- 15 files changed, 72 insertions(+), 80 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 0ec5db6c69d..2018015b46d 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -1,18 +1,19 @@ #include #include #include -#include -#include +#include #include #include -#include -#include -#include +#include #include -#include #include -#include +#include #include +#include +#include +#include +#include +#include namespace DB @@ -626,8 +627,7 @@ void ColumnAggregateFunction::getPermutation(PermutationSortDirection /*directio { size_t s = data.size(); res.resize(s); - for (size_t i = 0; i < s; ++i) - res[i] = i; + iota(res.data(), s, IColumn::Permutation::value_type(0)); } void ColumnAggregateFunction::updatePermutation(PermutationSortDirection, PermutationSortStability, diff --git a/src/Columns/ColumnConst.cpp b/src/Columns/ColumnConst.cpp index 10e960ea244..9aa0f5cfa49 100644 --- a/src/Columns/ColumnConst.cpp +++ b/src/Columns/ColumnConst.cpp @@ -2,9 +2,10 @@ #include #include -#include -#include #include +#include +#include +#include #include @@ -128,8 +129,7 @@ void ColumnConst::getPermutation(PermutationSortDirection /*direction*/, Permuta size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const { res.resize(s); - for (size_t i = 0; i < s; ++i) - res[i] = i; + iota(res.data(), s, IColumn::Permutation::value_type(0)); } void ColumnConst::updatePermutation(PermutationSortDirection /*direction*/, PermutationSortStability /*stability*/, diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index baccfc69147..20fc5d8e1fe 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -1,10 +1,11 @@ -#include #include -#include -#include -#include +#include #include #include +#include +#include +#include +#include #include @@ -163,8 +164,7 @@ void ColumnDecimal::getPermutation(IColumn::PermutationSortDirection directio if (limit >= data_size) limit = 0; - for (size_t i = 0; i < data_size; ++i) - res[i] = i; + iota(res.data(), data_size, IColumn::Permutation::value_type(0)); if constexpr (is_arithmetic_v && !is_big_int_v) { @@ -183,8 +183,7 @@ void ColumnDecimal::getPermutation(IColumn::PermutationSortDirection directio /// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters. if (data_size >= 256 && data_size <= std::numeric_limits::max() && use_radix_sort) { - for (size_t i = 0; i < data_size; ++i) - res[i] = i; + iota(res.data(), data_size, IColumn::Permutation::value_type(0)); bool try_sort = false; diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 057c0cd7112..02e6e9e56b4 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -1,11 +1,12 @@ -#include -#include #include +#include #include -#include -#include -#include +#include #include +#include +#include +#include +#include #include #include @@ -499,8 +500,7 @@ void ColumnSparse::getPermutationImpl(IColumn::PermutationSortDirection directio res.resize(_size); if (offsets->empty()) { - for (size_t i = 0; i < _size; ++i) - res[i] = i; + iota(res.data(), _size, IColumn::Permutation::value_type(0)); return; } diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index d8992125be4..356bb0493d2 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -1,16 +1,17 @@ #include -#include -#include #include +#include #include -#include +#include #include #include +#include +#include #include #include +#include #include -#include namespace DB @@ -378,8 +379,7 @@ void ColumnTuple::getPermutationImpl(IColumn::PermutationSortDirection direction { size_t rows = size(); res.resize(rows); - for (size_t i = 0; i < rows; ++i) - res[i] = i; + iota(res.data(), rows, IColumn::Permutation::value_type(0)); if (limit >= rows) limit = 0; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 37e62c76596..b1cf449dfde 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -1,24 +1,25 @@ #include "ColumnVector.h" -#include #include +#include #include #include -#include #include +#include +#include +#include +#include +#include #include #include #include #include #include #include -#include #include +#include #include -#include -#include -#include -#include +#include #include #include @@ -244,8 +245,7 @@ void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction if (limit >= data_size) limit = 0; - for (size_t i = 0; i < data_size; ++i) - res[i] = i; + iota(res.data(), data_size, IColumn::Permutation::value_type(0)); if constexpr (is_arithmetic_v && !is_big_int_v) { diff --git a/src/Columns/IColumnDummy.cpp b/src/Columns/IColumnDummy.cpp index 01091a87049..7c237536f94 100644 --- a/src/Columns/IColumnDummy.cpp +++ b/src/Columns/IColumnDummy.cpp @@ -1,7 +1,8 @@ -#include -#include -#include #include +#include +#include +#include +#include namespace DB @@ -87,8 +88,7 @@ void IColumnDummy::getPermutation(IColumn::PermutationSortDirection /*direction* size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const { res.resize(s); - for (size_t i = 0; i < s; ++i) - res[i] = i; + iota(res.data(), s, IColumn::Permutation::value_type(0)); } ColumnPtr IColumnDummy::replicate(const Offsets & offsets) const diff --git a/src/Columns/tests/gtest_column_stable_permutation.cpp b/src/Columns/tests/gtest_column_stable_permutation.cpp index df898cffa04..0dabd4d1fc2 100644 --- a/src/Columns/tests/gtest_column_stable_permutation.cpp +++ b/src/Columns/tests/gtest_column_stable_permutation.cpp @@ -9,7 +9,6 @@ #include #include #include - #include #include #include @@ -17,6 +16,7 @@ #include #include #include +#include using namespace DB; @@ -32,8 +32,7 @@ void stableGetColumnPermutation( size_t size = column.size(); out_permutation.resize(size); - for (size_t i = 0; i < size; ++i) - out_permutation[i] = i; + iota(out_permutation.data(), size, IColumn::Permutation::value_type(0)); std::stable_sort( out_permutation.begin(), @@ -146,10 +145,7 @@ void assertColumnPermutations(ColumnCreateFunc column_create_func, ValueTransfor std::vector> ranges(ranges_size); std::vector ranges_permutations(ranges_size); - for (size_t i = 0; i < ranges_size; ++i) - { - ranges_permutations[i] = i; - } + iota(ranges_permutations.data(), ranges_size, IColumn::Permutation::value_type(0)); IColumn::Permutation actual_permutation; IColumn::Permutation expected_permutation; diff --git a/src/Common/levenshteinDistance.cpp b/src/Common/levenshteinDistance.cpp index 9eb6c0f9050..3ab80af94bb 100644 --- a/src/Common/levenshteinDistance.cpp +++ b/src/Common/levenshteinDistance.cpp @@ -1,5 +1,6 @@ -#include #include +#include +#include namespace DB { @@ -11,8 +12,7 @@ size_t levenshteinDistance(const String & lhs, const String & rhs) PODArrayWithStackMemory row(n + 1); - for (size_t i = 1; i <= n; ++i) - row[i] = i; + iota(row.data() + 1, n, size_t(1)); for (size_t j = 1; j <= m; ++j) { diff --git a/src/Functions/FunctionsStringDistance.cpp b/src/Functions/FunctionsStringDistance.cpp index 3098d02630a..a5e819179d6 100644 --- a/src/Functions/FunctionsStringDistance.cpp +++ b/src/Functions/FunctionsStringDistance.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #ifdef __SSE4_2__ # include @@ -246,8 +247,7 @@ struct ByteEditDistanceImpl ResultType insertion = 0; ResultType deletion = 0; - for (size_t i = 0; i <= haystack_size; ++i) - distances0[i] = i; + iota(distances0.data(), haystack_size + 1, ResultType(0)); for (size_t pos_needle = 0; pos_needle < needle_size; ++pos_needle) { diff --git a/src/Functions/array/arraySort.cpp b/src/Functions/array/arraySort.cpp index a853289e8cc..184b1f82280 100644 --- a/src/Functions/array/arraySort.cpp +++ b/src/Functions/array/arraySort.cpp @@ -1,5 +1,6 @@ -#include #include +#include +#include namespace DB { @@ -55,9 +56,7 @@ ColumnPtr ArraySortImpl::execute( size_t size = offsets.size(); size_t nested_size = array.getData().size(); IColumn::Permutation permutation(nested_size); - - for (size_t i = 0; i < nested_size; ++i) - permutation[i] = i; + iota(permutation.data(), nested_size, IColumn::Permutation::value_type(0)); ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) diff --git a/src/Functions/rowNumberInBlock.cpp b/src/Functions/rowNumberInBlock.cpp index e5fe2aeb178..25c9e9c56f3 100644 --- a/src/Functions/rowNumberInBlock.cpp +++ b/src/Functions/rowNumberInBlock.cpp @@ -56,8 +56,7 @@ public: auto column = ColumnUInt64::create(); auto & data = column->getData(); data.resize(input_rows_count); - for (size_t i = 0; i < input_rows_count; ++i) - data[i] = i; + iota(data.data(), input_rows_count, UInt64(0)); return column; } diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index 89c4220ccdf..d75786f33b9 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #ifdef __SSE2__ #include @@ -155,8 +156,7 @@ void getBlockSortPermutationImpl(const Block & block, const SortDescription & de { size_t size = block.rows(); permutation.resize(size); - for (size_t i = 0; i < size; ++i) - permutation[i] = i; + iota(permutation.data(), size, IColumn::Permutation::value_type(0)); if (limit >= size) limit = 0; diff --git a/src/Processors/Transforms/PartialSortingTransform.cpp b/src/Processors/Transforms/PartialSortingTransform.cpp index 3fc9a4e71db..e79673f6645 100644 --- a/src/Processors/Transforms/PartialSortingTransform.cpp +++ b/src/Processors/Transforms/PartialSortingTransform.cpp @@ -1,7 +1,8 @@ -#include -#include #include +#include +#include #include +#include namespace DB { @@ -36,9 +37,7 @@ size_t getFilterMask(const ColumnRawPtrs & raw_block_columns, const Columns & th else { rows_to_compare.resize(num_rows); - - for (size_t i = 0; i < num_rows; ++i) - rows_to_compare[i] = i; + iota(rows_to_compare.data(), num_rows, UInt64(0)); size_t size = description.size(); for (size_t i = 0; i < size; ++i) diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index a0fabe3273c..46c6a77f60f 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -1,14 +1,12 @@ #include -#include -#include -#include "Core/UUID.h" #include +#include +#include #include #include #include #include -#include #include #include #include @@ -25,11 +23,14 @@ #include #include #include -#include #include #include +#include #include #include +#include +#include +#include namespace DB { @@ -619,8 +620,7 @@ void QueryPipelineBuilder::addPipelineBefore(QueryPipelineBuilder pipeline) bool has_extremes = pipe.getExtremesPort(); size_t num_extra_ports = (has_totals ? 1 : 0) + (has_extremes ? 1 : 0); IProcessor::PortNumbers delayed_streams(pipe.numOutputPorts() + num_extra_ports); - for (size_t i = 0; i < delayed_streams.size(); ++i) - delayed_streams[i] = i; + iota(delayed_streams.data(), delayed_streams.size(), IProcessor::PortNumbers::value_type(0)); auto * collected_processors = pipe.collected_processors; From 22ef5443bb0d0c2a1e6c2fa2b178765dc3cb761b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 29 Dec 2023 14:44:16 +0100 Subject: [PATCH 17/61] Move findNumeric to .cpp --- .../AggregateFunctionMax.cpp | 10 +-- .../AggregateFunctionMin.cpp | 10 +-- src/AggregateFunctions/findNumeric.cpp | 15 ----- .../findNumeric.h => Common/findExtreme.cpp} | 65 ++++++++----------- src/Common/findExtreme.h | 45 +++++++++++++ 5 files changed, 82 insertions(+), 63 deletions(-) delete mode 100644 src/AggregateFunctions/findNumeric.cpp rename src/{AggregateFunctions/findNumeric.h => Common/findExtreme.cpp} (57%) create mode 100644 src/Common/findExtreme.h diff --git a/src/AggregateFunctions/AggregateFunctionMax.cpp b/src/AggregateFunctions/AggregateFunctionMax.cpp index a440aedb62c..3d4d23136a1 100644 --- a/src/AggregateFunctions/AggregateFunctionMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionMax.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include namespace DB { @@ -53,10 +53,10 @@ void AggregateFunctionsSingleValueMax= 0) \ { \ const auto & flags = assert_cast(*columns[if_argument_pos]).getData(); \ - opt = findNumericMaxIf(column.getData().data(), flags.data(), row_begin, row_end); \ + opt = findExtremeMaxIf(column.getData().data(), flags.data(), row_begin, row_end); \ } \ else \ - opt = findNumericMax(column.getData().data(), row_begin, row_end); \ + opt = findExtremeMax(column.getData().data(), row_begin, row_end); \ if (opt.has_value()) \ this->data(place).changeIfGreater(opt.value()); \ } @@ -140,10 +140,10 @@ void AggregateFunctionsSingleValueMax(row_end); \ for (size_t i = row_begin; i < row_end; ++i) \ final_flags[i] = (!null_map[i]) & !!if_flags[i]; \ - opt = findNumericMaxIf(column.getData().data(), final_flags.get(), row_begin, row_end); \ + opt = findExtremeMaxIf(column.getData().data(), final_flags.get(), row_begin, row_end); \ } \ else \ - opt = findNumericMaxNotNull(column.getData().data(), null_map, row_begin, row_end); \ + opt = findExtremeMaxNotNull(column.getData().data(), null_map, row_begin, row_end); \ if (opt.has_value()) \ this->data(place).changeIfGreater(opt.value()); \ } diff --git a/src/AggregateFunctions/AggregateFunctionMin.cpp b/src/AggregateFunctions/AggregateFunctionMin.cpp index 8d5d12fa626..02d041ad12b 100644 --- a/src/AggregateFunctions/AggregateFunctionMin.cpp +++ b/src/AggregateFunctions/AggregateFunctionMin.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include namespace DB @@ -54,10 +54,10 @@ public: if (if_argument_pos >= 0) \ { \ const auto & flags = assert_cast(*columns[if_argument_pos]).getData(); \ - opt = findNumericMinIf(column.getData().data(), flags.data(), row_begin, row_end); \ + opt = findExtremeMinIf(column.getData().data(), flags.data(), row_begin, row_end); \ } \ else \ - opt = findNumericMin(column.getData().data(), row_begin, row_end); \ + opt = findExtremeMin(column.getData().data(), row_begin, row_end); \ if (opt.has_value()) \ this->data(place).changeIfLess(opt.value()); \ } @@ -141,10 +141,10 @@ void AggregateFunctionsSingleValueMin::addBatchSinglePlace( auto final_flags = std::make_unique(row_end); \ for (size_t i = row_begin; i < row_end; ++i) \ final_flags[i] = (!null_map[i]) & !!if_flags[i]; \ - opt = findNumericMinIf(column.getData().data(), final_flags.get(), row_begin, row_end); \ + opt = findExtremeMinIf(column.getData().data(), final_flags.get(), row_begin, row_end); \ } \ else \ - opt = findNumericMinNotNull(column.getData().data(), null_map, row_begin, row_end); \ + opt = findExtremeMinNotNull(column.getData().data(), null_map, row_begin, row_end); \ if (opt.has_value()) \ this->data(place).changeIfLess(opt.value()); \ } diff --git a/src/AggregateFunctions/findNumeric.cpp b/src/AggregateFunctions/findNumeric.cpp deleted file mode 100644 index bbad8c1fe3d..00000000000 --- a/src/AggregateFunctions/findNumeric.cpp +++ /dev/null @@ -1,15 +0,0 @@ -#include - -namespace DB -{ -#define INSTANTIATION(T) \ - template std::optional findNumericMin(const T * __restrict ptr, size_t start, size_t end); \ - template std::optional findNumericMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ - template std::optional findNumericMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ - template std::optional findNumericMax(const T * __restrict ptr, size_t start, size_t end); \ - template std::optional findNumericMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ - template std::optional findNumericMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); - -FOR_BASIC_NUMERIC_TYPES(INSTANTIATION) -#undef INSTANTIATION -} diff --git a/src/AggregateFunctions/findNumeric.h b/src/Common/findExtreme.cpp similarity index 57% rename from src/AggregateFunctions/findNumeric.h rename to src/Common/findExtreme.cpp index df7c325569a..e1f1e199d56 100644 --- a/src/AggregateFunctions/findNumeric.h +++ b/src/Common/findExtreme.cpp @@ -1,18 +1,9 @@ -#pragma once - #include -#include -#include -#include #include - -#include -#include +#include namespace DB { -template -concept is_any_native_number = (is_any_of); template struct MinComparator @@ -28,7 +19,7 @@ struct MaxComparator MULTITARGET_FUNCTION_AVX2_SSE42( MULTITARGET_FUNCTION_HEADER(template static std::optional NO_INLINE), - findNumericExtremeImpl, + findExtremeImpl, MULTITARGET_FUNCTION_BODY((const T * __restrict ptr, const UInt8 * __restrict condition_map [[maybe_unused]], size_t row_begin, size_t row_end) { size_t count = row_end - row_begin; @@ -86,69 +77,67 @@ MULTITARGET_FUNCTION_AVX2_SSE42( } )) - /// Given a vector of T finds the extreme (MIN or MAX) value template static std::optional -findNumericExtreme(const T * __restrict ptr, const UInt8 * __restrict condition_map [[maybe_unused]], size_t start, size_t end) +findExtreme(const T * __restrict ptr, const UInt8 * __restrict condition_map [[maybe_unused]], size_t start, size_t end) { #if USE_MULTITARGET_CODE /// We see no benefit from using AVX512BW or AVX512F (over AVX2), so we only declare SSE and AVX2 if (isArchSupported(TargetArch::AVX2)) - return findNumericExtremeImplAVX2(ptr, condition_map, start, end); + return findExtremeImplAVX2(ptr, condition_map, start, end); if (isArchSupported(TargetArch::SSE42)) - return findNumericExtremeImplSSE42(ptr, condition_map, start, end); + return findExtremeImplSSE42(ptr, condition_map, start, end); #endif - return findNumericExtremeImpl(ptr, condition_map, start, end); + return findExtremeImpl(ptr, condition_map, start, end); } template -std::optional findNumericMin(const T * __restrict ptr, size_t start, size_t end) +std::optional findExtremeMin(const T * __restrict ptr, size_t start, size_t end) { - return findNumericExtreme, true, false>(ptr, nullptr, start, end); + return findExtreme, true, false>(ptr, nullptr, start, end); } template -std::optional findNumericMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end) +std::optional findExtremeMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end) { - return findNumericExtreme, false, true>(ptr, condition_map, start, end); + return findExtreme, false, true>(ptr, condition_map, start, end); } template -std::optional findNumericMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end) +std::optional findExtremeMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end) { - return findNumericExtreme, false, false>(ptr, condition_map, start, end); + return findExtreme, false, false>(ptr, condition_map, start, end); } template -std::optional findNumericMax(const T * __restrict ptr, size_t start, size_t end) +std::optional findExtremeMax(const T * __restrict ptr, size_t start, size_t end) { - return findNumericExtreme, true, false>(ptr, nullptr, start, end); + return findExtreme, true, false>(ptr, nullptr, start, end); } template -std::optional findNumericMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end) +std::optional findExtremeMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end) { - return findNumericExtreme, false, true>(ptr, condition_map, start, end); + return findExtreme, false, true>(ptr, condition_map, start, end); } template -std::optional findNumericMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end) +std::optional findExtremeMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end) { - return findNumericExtreme, false, false>(ptr, condition_map, start, end); + return findExtreme, false, false>(ptr, condition_map, start, end); } -#define EXTERN_INSTANTIATION(T) \ - extern template std::optional findNumericMin(const T * __restrict ptr, size_t start, size_t end); \ - extern template std::optional findNumericMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ - extern template std::optional findNumericMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ - extern template std::optional findNumericMax(const T * __restrict ptr, size_t start, size_t end); \ - extern template std::optional findNumericMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ - extern template std::optional findNumericMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); - - FOR_BASIC_NUMERIC_TYPES(EXTERN_INSTANTIATION) -#undef EXTERN_INSTANTIATION +#define INSTANTIATION(T) \ + template std::optional findExtremeMin(const T * __restrict ptr, size_t start, size_t end); \ + template std::optional findExtremeMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ + template std::optional findExtremeMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ + template std::optional findExtremeMax(const T * __restrict ptr, size_t start, size_t end); \ + template std::optional findExtremeMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ + template std::optional findExtremeMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); +FOR_BASIC_NUMERIC_TYPES(INSTANTIATION) +#undef INSTANTIATION } diff --git a/src/Common/findExtreme.h b/src/Common/findExtreme.h new file mode 100644 index 00000000000..b38c24697c0 --- /dev/null +++ b/src/Common/findExtreme.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include +#include + +#include +#include + +namespace DB +{ +template +concept is_any_native_number = (is_any_of); + +template +std::optional findExtremeMin(const T * __restrict ptr, size_t start, size_t end); + +template +std::optional findExtremeMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); + +template +std::optional findExtremeMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); + +template +std::optional findExtremeMax(const T * __restrict ptr, size_t start, size_t end); + +template +std::optional findExtremeMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); + +template +std::optional findExtremeMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); + +#define EXTERN_INSTANTIATION(T) \ + extern template std::optional findExtremeMin(const T * __restrict ptr, size_t start, size_t end); \ + extern template std::optional findExtremeMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ + extern template std::optional findExtremeMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ + extern template std::optional findExtremeMax(const T * __restrict ptr, size_t start, size_t end); \ + extern template std::optional findExtremeMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ + extern template std::optional findExtremeMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); + + FOR_BASIC_NUMERIC_TYPES(EXTERN_INSTANTIATION) +#undef EXTERN_INSTANTIATION + +} From 5fb7f9f861ea5adaece97c1afbd4ba1283957049 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 29 Dec 2023 18:09:20 +0100 Subject: [PATCH 18/61] Ignore other numeric types for now --- src/AggregateFunctions/AggregateFunctionMax.cpp | 14 ++++++++++++++ src/AggregateFunctions/AggregateFunctionMin.cpp | 14 ++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionMax.cpp b/src/AggregateFunctions/AggregateFunctionMax.cpp index 3d4d23136a1..2577c932592 100644 --- a/src/AggregateFunctions/AggregateFunctionMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionMax.cpp @@ -74,6 +74,13 @@ void AggregateFunctionsSingleValueMax::addBatchSinglePlace( Arena * arena, ssize_t if_argument_pos) const { + if constexpr (!std::is_same_v || !std::is_same_v) + { + /// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's + /// faster than doing a permutation + return Parent::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos); + } + constexpr int nan_direction_hint = 1; auto const & column = *columns[0]; if (if_argument_pos >= 0) @@ -162,6 +169,13 @@ void AggregateFunctionsSingleValueMax::addBatchSinglePlaceNotNull( Arena * arena, ssize_t if_argument_pos) const { + if constexpr (!std::is_same_v || !std::is_same_v) + { + /// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's + /// faster than doing a permutation + return Parent::addBatchSinglePlaceNotNull(row_begin, row_end, place, columns, null_map, arena, if_argument_pos); + } + constexpr int nan_direction_hint = 1; auto const & column = *columns[0]; if (if_argument_pos >= 0) diff --git a/src/AggregateFunctions/AggregateFunctionMin.cpp b/src/AggregateFunctions/AggregateFunctionMin.cpp index 02d041ad12b..701101e7207 100644 --- a/src/AggregateFunctions/AggregateFunctionMin.cpp +++ b/src/AggregateFunctions/AggregateFunctionMin.cpp @@ -75,6 +75,13 @@ void AggregateFunctionsSingleValueMin::addBatchSinglePlace( Arena * arena, ssize_t if_argument_pos) const { + if constexpr (!std::is_same_v || !std::is_same_v) + { + /// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's + /// faster than doing a permutation + return Parent::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos); + } + constexpr int nan_direction_hint = 1; auto const & column = *columns[0]; if (if_argument_pos >= 0) @@ -163,6 +170,13 @@ void AggregateFunctionsSingleValueMin::addBatchSinglePlaceNotNull( Arena * arena, ssize_t if_argument_pos) const { + if constexpr (!std::is_same_v || !std::is_same_v) + { + /// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's + /// faster than doing a permutation + return Parent::addBatchSinglePlaceNotNull(row_begin, row_end, place, columns, null_map, arena, if_argument_pos); + } + constexpr int nan_direction_hint = 1; auto const & column = *columns[0]; if (if_argument_pos >= 0) From b95bdef09ee9474193beaba8c6eab078bb9970eb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 29 Dec 2023 17:41:11 +0000 Subject: [PATCH 19/61] Update StorageS3 and StorageS3Cluster --- src/Storages/HDFS/StorageHDFS.cpp | 17 +-- src/Storages/S3Queue/StorageS3Queue.cpp | 117 +++++++++++++++--- src/Storages/S3Queue/StorageS3Queue.h | 11 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageS3.cpp | 152 ++++++++++++++++++------ src/Storages/StorageS3.h | 14 ++- src/Storages/VirtualColumnUtils.cpp | 6 +- src/Storages/VirtualColumnUtils.h | 2 +- 8 files changed, 238 insertions(+), 85 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 9d719413c8d..fe37b2eb57a 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -417,7 +417,7 @@ public: uris = getPathsList(path_from_uri, uri_without_path, context); ActionsDAGPtr filter_dag; if (!uris.empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, uris[0].path); + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); if (filter_dag) { @@ -492,7 +492,7 @@ public: { ActionsDAGPtr filter_dag; if (!uris.empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, getPathFromUriAndUriWithoutPath(uris[0]).first); + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); if (filter_dag) { @@ -893,9 +893,6 @@ public: ReadFromFormatInfo info_, bool need_only_count_, std::shared_ptr storage_, - // StorageSnapshotPtr storage_snapshot_, - // const StorageEmbeddedRocksDB & storage_, - // SelectQueryInfo query_info_, ContextPtr context_, size_t max_block_size_, size_t num_streams_) @@ -907,9 +904,6 @@ public: , info(std::move(info_)) , need_only_count(need_only_count_) , storage(std::move(storage_)) - // , storage_snapshot(std::move(storage_snapshot_)) - // , storage(storage_) - // , query_info(std::move(query_info_)) , context(std::move(context_)) , max_block_size(max_block_size_) , num_streams(num_streams_) @@ -925,19 +919,12 @@ private: const bool need_only_count; std::shared_ptr storage; - // StorageSnapshotPtr storage_snapshot; - // const StorageEmbeddedRocksDB & storage; - // SelectQueryInfo query_info; ContextPtr context; - size_t max_block_size; size_t num_streams; std::shared_ptr iterator_wrapper; - // FieldVectorPtr keys; - // bool all_scan = false; - void createIterator(const ActionsDAG::Node * predicate); }; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 33e63d45c8d..1a6666c00d0 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -1,3 +1,6 @@ +#include "Processors/QueryPlan/QueryPlan.h" +#include "Processors/QueryPlan/SourceStepWithFilter.h" +#include "QueryPipeline/QueryPipelineBuilder.h" #include "config.h" #if USE_AWS_S3 @@ -204,10 +207,71 @@ bool StorageS3Queue::supportsSubsetOfColumns(const ContextPtr & context_) const return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context_, format_settings); } -Pipe StorageS3Queue::read( +class ReadFromS3Queue : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromS3Queue"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + void applyFilters() override; + + ReadFromS3Queue( + Block sample_block, + ReadFromFormatInfo info_, + std::shared_ptr storage_, + // StorageSnapshotPtr storage_snapshot_, + // Names column_names_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_) + : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) + , info(std::move(info_)) + , storage(std::move(storage_)) + // , storage_snapshot(std::move(storage_snapshot_)) + // , column_names(std::move(column_names_)) + , context(std::move(context_)) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + { + } + +private: + ReadFromFormatInfo info; + std::shared_ptr storage; + // StorageSnapshotPtr storage_snapshot; + // Names column_names; + ContextPtr context; + size_t max_block_size; + size_t num_streams; + + std::shared_ptr iterator; + + void createIterator(const ActionsDAG::Node * predicate); +}; + +void ReadFromS3Queue::createIterator(const ActionsDAG::Node * predicate) +{ + if (iterator) + return; + + iterator = storage->createFileIterator(context, predicate); +} + + +void ReadFromS3Queue::applyFilters() +{ + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + createIterator(predicate); +} + +void StorageS3Queue::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, + SelectQueryInfo & /*query_info*/, ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, @@ -225,27 +289,43 @@ Pipe StorageS3Queue::read( "Cannot read from {} with attached materialized views", getName()); } - Pipes pipes; - const size_t adjusted_num_streams = std::min(num_streams, s3queue_settings->s3queue_processing_threads_num); + auto this_ptr = std::static_pointer_cast(shared_from_this()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); - auto file_iterator = createFileIterator(local_context, query_info.query); + auto reading = std::make_unique( + read_from_format_info.source_header, + read_from_format_info, + std::move(this_ptr), + // storage_snapshot, + // column_names, + local_context, + max_block_size, + num_streams); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromS3Queue::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + Pipes pipes; + const size_t adjusted_num_streams = std::min(num_streams, storage->s3queue_settings->s3queue_processing_threads_num); + + createIterator(nullptr); for (size_t i = 0; i < adjusted_num_streams; ++i) - pipes.emplace_back(createSource(file_iterator, column_names, storage_snapshot, max_block_size, local_context)); - return Pipe::unitePipes(std::move(pipes)); + pipes.emplace_back(storage->createSource(info, iterator, max_block_size, context)); + pipeline.init(Pipe::unitePipes(std::move(pipes))); } std::shared_ptr StorageS3Queue::createSource( + const ReadFromFormatInfo & info, std::shared_ptr file_iterator, - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, size_t max_block_size, ContextPtr local_context) { auto configuration_snapshot = updateConfigurationAndGetCopy(local_context); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); auto internal_source = std::make_unique( - read_from_format_info, configuration.format, getName(), local_context, format_settings, + info, configuration.format, getName(), local_context, format_settings, max_block_size, configuration_snapshot.request_settings, configuration_snapshot.compression_method, @@ -253,7 +333,7 @@ std::shared_ptr StorageS3Queue::createSource( configuration_snapshot.url.bucket, configuration_snapshot.url.version_id, configuration_snapshot.url.uri.getHost() + std::to_string(configuration_snapshot.url.uri.getPort()), - file_iterator, local_context->getSettingsRef().max_download_threads, false, /* query_info */ std::nullopt); + file_iterator, local_context->getSettingsRef().max_download_threads, false); auto file_deleter = [this, bucket = configuration_snapshot.url.bucket, client = configuration_snapshot.client, blob_storage_log = BlobStorageLogWriter::create()](const std::string & path) mutable { @@ -277,8 +357,8 @@ std::shared_ptr StorageS3Queue::createSource( }; auto s3_queue_log = s3queue_settings->s3queue_enable_logging_to_s3queue_log ? local_context->getS3QueueLog() : nullptr; return std::make_shared( - getName(), read_from_format_info.source_header, std::move(internal_source), - files_metadata, after_processing, file_deleter, read_from_format_info.requested_virtual_columns, + getName(), info.source_header, std::move(internal_source), + files_metadata, after_processing, file_deleter, info.requested_virtual_columns, local_context, shutdown_called, table_is_being_dropped, s3_queue_log, getStorageID(), log); } @@ -375,13 +455,14 @@ bool StorageS3Queue::streamToViews() auto block_io = interpreter.execute(); auto file_iterator = createFileIterator(s3queue_context, nullptr); + auto read_from_format_info = prepareReadingFromFormat(block_io.pipeline.getHeader().getNames(), storage_snapshot, supportsSubsetOfColumns(s3queue_context), getVirtuals()); + Pipes pipes; pipes.reserve(s3queue_settings->s3queue_processing_threads_num); for (size_t i = 0; i < s3queue_settings->s3queue_processing_threads_num; ++i) { auto source = createSource( - file_iterator, block_io.pipeline.getHeader().getNames(), - storage_snapshot, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); + read_from_format_info, file_iterator, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); pipes.emplace_back(std::move(source)); } @@ -479,10 +560,10 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const } } -std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query) +std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, const ActionsDAG::Node * predicate) { auto glob_iterator = std::make_unique( - *configuration.client, configuration.url, query, virtual_columns, local_context, + *configuration.client, configuration.url, predicate, virtual_columns, local_context, /* read_keys */nullptr, configuration.request_settings); return std::make_shared(files_metadata, std::move(glob_iterator), shutdown_called); } diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index f26b1175150..3d3594dc2ab 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -39,10 +39,11 @@ public: String getName() const override { return "S3Queue"; } - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, + SelectQueryInfo & /*query_info*/, ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, @@ -57,6 +58,7 @@ public: zkutil::ZooKeeperPtr getZooKeeper() const; private: + friend class ReadFromS3Queue; using FileIterator = StorageS3QueueSource::FileIterator; const std::unique_ptr s3queue_settings; @@ -85,11 +87,10 @@ private: bool supportsSubsetOfColumns(const ContextPtr & context_) const; bool supportsSubcolumns() const override { return true; } - std::shared_ptr createFileIterator(ContextPtr local_context, ASTPtr query); + std::shared_ptr createFileIterator(ContextPtr local_context, const ActionsDAG::Node * predicate); std::shared_ptr createSource( + const ReadFromFormatInfo & info, std::shared_ptr file_iterator, - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, size_t max_block_size, ContextPtr local_context); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index b040f452410..e4619d64ae3 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -960,8 +960,8 @@ StorageFileSource::FilesIterator::FilesIterator( : files(files_), archive_info(std::move(archive_info_)), distributed_processing(distributed_processing_), context(context_) { ActionsDAGPtr filter_dag; - if (!distributed_processing && !archive_info && !files.empty() && !files[0].empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, files[0]); + if (!distributed_processing && !archive_info && !files.empty()) + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); if (filter_dag) VirtualColumnUtils::filterByPathOrFile(files, files, filter_dag, virtual_columns, context_); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 096e2e88f91..780a2755bcf 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -159,6 +159,8 @@ public: , max_block_size(max_block_size_) , num_streams(num_streams_) { + query_configuration = storage.updateConfigurationAndGetCopy(local_context); + virtual_columns = storage.getVirtuals(); } private: @@ -166,10 +168,17 @@ private: StorageSnapshotPtr storage_snapshot; StorageS3 & storage; SelectQueryInfo query_info; + StorageS3::Configuration query_configuration; + NamesAndTypesList virtual_columns; + ContextPtr local_context; size_t max_block_size; size_t num_streams; + + std::shared_ptr iterator_wrapper; + + void createIterator(const ActionsDAG::Node * predicate); }; @@ -231,24 +240,14 @@ static std::vector filterKeysForPartitionPruning( const std::vector & keys, const String & bucket, const NamesAndTypesList & virtual_columns, - const std::vector & filter_dags, + const ActionsDAG::Node * predicate, ContextPtr context) { std::unordered_set result_keys(keys.begin(), keys.end()); - for (const auto & filter_dag : filter_dags) - { - if (result_keys.empty()) - break; - auto block = getBlockWithVirtuals(virtual_columns, bucket, result_keys); - - auto filter_actions = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), block); - if (!filter_actions) - continue; - VirtualColumnUtils::filterBlockWithDAG(filter_actions, block, context); - - result_keys = VirtualColumnUtils::extractSingleValueFromBlock(block, "_key"); - } + auto block = getBlockWithVirtuals(virtual_columns, bucket, result_keys); + VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); + result_keys = VirtualColumnUtils::extractSingleValueFromBlock(block, "_key"); LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Applied partition pruning {} from {} keys left", result_keys.size(), keys.size()); return std::vector(result_keys.begin(), result_keys.end()); @@ -309,6 +308,57 @@ public: fillInternalBufferAssumeLocked(); } + Impl( + const S3::Client & client_, + const S3::URI & globbed_uri_, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns_, + ContextPtr context_, + KeysWithInfo * read_keys_, + const S3Settings::RequestSettings & request_settings_, + std::function file_progress_callback_) + : WithContext(context_) + , client(client_.clone()) + , globbed_uri(globbed_uri_) + , virtual_columns(virtual_columns_) + , read_keys(read_keys_) + , request_settings(request_settings_) + , list_objects_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, CurrentMetrics::StorageS3ThreadsScheduled, 1) + , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, "ListObjects")) + , file_progress_callback(file_progress_callback_) + { + if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) + throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Expression can not have wildcards inside bucket name"); + + const String key_prefix = globbed_uri.key.substr(0, globbed_uri.key.find_first_of("*?{")); + + /// We don't have to list bucket, because there is no asterisks. + if (key_prefix.size() == globbed_uri.key.size()) + { + buffer.emplace_back(std::make_shared(globbed_uri.key, std::nullopt)); + buffer_iter = buffer.begin(); + is_finished = true; + return; + } + + request.SetBucket(globbed_uri.bucket); + request.SetPrefix(key_prefix); + request.SetMaxKeys(static_cast(request_settings.list_object_keys_size)); + + outcome_future = listObjectsAsync(); + + matcher = std::make_unique(makeRegexpPatternFromGlobs(globbed_uri.key)); + if (!matcher->ok()) + throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, + "Cannot compile regex from glob ({}): {}", globbed_uri.key, matcher->error()); + + recursive = globbed_uri.key == "/**" ? true : false; + fillInternalBufferAssumeLocked(); + + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + is_initialized = true; + } + KeyWithInfoPtr next() { std::lock_guard lock(mutex); @@ -439,6 +489,15 @@ private: VirtualColumnUtils::filterByPathOrFile(temp_buffer, paths, query, virtual_columns, getContext(), filter_ast); } + else if (filter_dag) + { + std::vector paths; + paths.reserve(temp_buffer.size()); + for (const auto & key_with_info : temp_buffer) + paths.push_back(fs::path(globbed_uri.bucket) / key_with_info->key); + + VirtualColumnUtils::filterByPathOrFile(temp_buffer, paths, filter_dag, virtual_columns, getContext()); + } buffer = std::move(temp_buffer); @@ -481,6 +540,7 @@ private: NamesAndTypesList virtual_columns; bool is_initialized{false}; ASTPtr filter_ast; + ActionsDAGPtr filter_dag; std::unique_ptr matcher; bool recursive{false}; bool is_finished{false}; @@ -508,6 +568,19 @@ StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( { } +StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( + const S3::Client & client_, + const S3::URI & globbed_uri_, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns_, + ContextPtr context, + KeysWithInfo * read_keys_, + const S3Settings::RequestSettings & request_settings_, + std::function file_progress_callback_) + : pimpl(std::make_shared(client_, globbed_uri_, predicate, virtual_columns_, context, read_keys_, request_settings_, file_progress_callback_)) +{ +} + StorageS3Source::KeyWithInfoPtr StorageS3Source::DisclosedGlobIterator::next() { return pimpl->next(); @@ -646,8 +719,7 @@ StorageS3Source::StorageS3Source( const String & url_host_and_port_, std::shared_ptr file_iterator_, const size_t max_parsing_threads_, - bool need_only_count_, - std::optional query_info_) + bool need_only_count_) : SourceWithKeyCondition(info.source_header, false) , WithContext(context_) , name(std::move(name_)) @@ -663,7 +735,6 @@ StorageS3Source::StorageS3Source( , client(client_) , sample_block(info.format_header) , format_settings(format_settings_) - , query_info(std::move(query_info_)) , requested_virtual_columns(info.requested_virtual_columns) , file_iterator(file_iterator_) , max_parsing_threads(max_parsing_threads_) @@ -1151,8 +1222,7 @@ static std::shared_ptr createFileIterator( const StorageS3::Configuration & configuration, bool distributed_processing, ContextPtr local_context, - ASTPtr query, - const std::vector & filter_dags, + const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, StorageS3::KeysWithInfo * read_keys = nullptr, std::function file_progress_callback = {}) @@ -1165,12 +1235,12 @@ static std::shared_ptr createFileIterator( { /// Iterate through disclosed globs and make a source for each file return std::make_shared( - *configuration.client, configuration.url, query, virtual_columns, + *configuration.client, configuration.url, predicate, virtual_columns, local_context, read_keys, configuration.request_settings, file_progress_callback); } else { - Strings keys = filterKeysForPartitionPruning(configuration.keys, configuration.url.bucket, virtual_columns, filter_dags, local_context); + Strings keys = filterKeysForPartitionPruning(configuration.keys, configuration.url.bucket, virtual_columns, predicate, local_context); return std::make_shared( *configuration.client, configuration.url.version_id, keys, configuration.url.bucket, configuration.request_settings, read_keys, file_progress_callback); @@ -1217,19 +1287,34 @@ void StorageS3::read( query_plan.addStep(std::move(reading)); } +void ReadFromStorageS3Step::applyFilters() +{ + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, local_context); + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + createIterator(predicate); +} + +void ReadFromStorageS3Step::createIterator(const ActionsDAG::Node * predicate) +{ + if (iterator_wrapper) + return; + + iterator_wrapper = createFileIterator( + query_configuration, storage.distributed_processing, local_context, predicate, + virtual_columns, nullptr, local_context->getFileProgressCallback()); +} + void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - auto query_configuration = storage.updateConfigurationAndGetCopy(local_context); - if (storage.partition_by && query_configuration.withWildcard()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet"); - auto virtual_columns = storage.getVirtuals(); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, storage.supportsSubsetOfColumns(local_context), virtual_columns); + createIterator(nullptr); - std::shared_ptr iterator_wrapper = createFileIterator( - query_configuration, storage.distributed_processing, local_context, query_info.query, filter_dags, - virtual_columns, nullptr, local_context->getFileProgressCallback()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, storage.supportsSubsetOfColumns(local_context), virtual_columns); size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); if (estimated_keys_count > 1) @@ -1264,19 +1349,12 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, query_configuration.url.uri.getHost() + std::to_string(query_configuration.url.uri.getPort()), iterator_wrapper, max_parsing_threads, - need_only_count, - query_info)); + need_only_count)); } pipeline.init(Pipe::unitePipes(std::move(pipes))); } - -void ReadFromStorageS3Step::applyFilters() -{ - /// We will use filter_dags in filterKeysForPartitionPruning called from initializePipeline, nothing to do here -} - SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { auto query_configuration = updateConfigurationAndGetCopy(local_context); @@ -1853,7 +1931,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( { KeysWithInfo read_keys; - auto file_iterator = createFileIterator(configuration, false, ctx, {}, {}, {}, &read_keys); + auto file_iterator = createFileIterator(configuration, false, ctx, {}, {}, &read_keys); ReadBufferIterator read_buffer_iterator(file_iterator, read_keys, configuration, format_settings, ctx); return readSchemaFromFormat(configuration.format, format_settings, read_buffer_iterator, configuration.withGlobs(), ctx); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 07d965d8bb3..dd7e0edb2d9 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -85,6 +85,16 @@ public: const S3Settings::RequestSettings & request_settings_ = {}, std::function progress_callback_ = {}); + DisclosedGlobIterator( + const S3::Client & client_, + const S3::URI & globbed_uri_, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns, + ContextPtr context, + KeysWithInfo * read_keys_ = nullptr, + const S3Settings::RequestSettings & request_settings_ = {}, + std::function progress_callback_ = {}); + KeyWithInfoPtr next() override; size_t estimatedKeysCount() override; @@ -145,8 +155,7 @@ public: const String & url_host_and_port, std::shared_ptr file_iterator_, size_t max_parsing_threads, - bool need_only_count_, - std::optional query_info); + bool need_only_count_); ~StorageS3Source() override; @@ -180,7 +189,6 @@ private: std::shared_ptr client; Block sample_block; std::optional format_settings; - std::optional query_info; struct ReaderHolder { diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 7690e160255..b63b4e7cca7 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -390,7 +390,7 @@ static void addPathAndFileToVirtualColumns(Block & block, const String & path, s block.getByName("_idx").column->assumeMutableRef().insert(idx); } -ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const String & path_example) +ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns) { if (!predicate || virtual_columns.empty()) return {}; @@ -401,10 +401,8 @@ ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, con if (column.name == "_file" || column.name == "_path") block.insert({column.type->createColumn(), column.type, column.name}); } - /// Create a block with one row to construct filter - /// Append "idx" column as the filter result + block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); - addPathAndFileToVirtualColumns(block, path_example, 0); return splitFilterDagForAllowedInputs(predicate, block); } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 4f9636b4213..6e1af0995cc 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -77,7 +77,7 @@ void filterByPathOrFile(std::vector & sources, const std::vector & pa sources = std::move(filtered_sources); } -ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const String & path_example); +ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context); From 47c3696a46a19674e04c6a1099e0a1429b90933e Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Fri, 29 Dec 2023 20:41:33 +0100 Subject: [PATCH 20/61] Fix build --- src/Interpreters/DDLTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index e7796c5d3a5..85bf6fec655 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -221,7 +221,7 @@ bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, c std::exception_ptr first_exception = nullptr; const auto maybe_secure_port = global_context->getTCPPortSecure(); - const auto port = global_context->getTCPPort() + const auto port = global_context->getTCPPort(); if (config_host_name) { From b7cc6d4615c6ce47c695962747044fb1e49c099b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 2 Jan 2024 13:08:04 +0000 Subject: [PATCH 21/61] Fixing tests. --- src/Storages/StorageFile.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index e4619d64ae3..12a8eed106e 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1517,7 +1517,16 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui need_only_count)); } - pipeline.init(Pipe::unitePipes(std::move(pipes))); + auto pipe = Pipe::unitePipes(std::move(pipes)); + size_t output_ports = pipe.numOutputPorts(); + const bool parallelize_output = context->getSettingsRef().parallelize_output_from_storages; + if (parallelize_output && storage->parallelizeOutputAfterReading(context) && output_ports > 0 && output_ports < max_num_streams) + pipe.resize(max_num_streams); + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); } From 0f76967f9755d3b15eb530a1a6e2dc00e653b9d9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 2 Jan 2024 13:45:41 +0000 Subject: [PATCH 22/61] Add reading step to Azure. --- src/Storages/StorageAzureBlob.cpp | 207 ++++++++++++++++++++++++------ src/Storages/StorageAzureBlob.h | 24 +++- 2 files changed, 188 insertions(+), 43 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 9564bad485c..048248ef334 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1,4 +1,6 @@ #include +#include "Processors/QueryPlan/QueryPlan.h" +#include "Processors/QueryPlan/SourceStepWithFilter.h" #if USE_AZURE_BLOB_STORAGE @@ -666,7 +668,58 @@ private: } -Pipe StorageAzureBlob::read( +class ReadFromAzureBlob : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromAzureBlob"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + void applyFilters() override; + + ReadFromAzureBlob( + Block sample_block, + std::shared_ptr storage_, + ReadFromFormatInfo info_, + const bool need_only_count_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_) + : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) + , storage(std::move(storage_)) + , info(std::move(info_)) + , need_only_count(need_only_count_) + , context(std::move(context_)) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + { + } + +private: + std::shared_ptr storage; + ReadFromFormatInfo info; + const bool need_only_count; + + ContextPtr context; + + size_t max_block_size; + const size_t num_streams; + + std::shared_ptr iterator_wrapper; + + void createIterator(const ActionsDAG::Node * predicate); +}; + +void ReadFromAzureBlob::applyFilters() +{ + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + createIterator(predicate); +} + +void StorageAzureBlob::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -678,51 +731,76 @@ Pipe StorageAzureBlob::read( if (partition_by && configuration.withWildcard()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned Azure storage is not implemented yet"); - Pipes pipes; - - std::shared_ptr iterator_wrapper; - if (distributed_processing) - { - iterator_wrapper = std::make_shared(local_context, - local_context->getReadTaskCallback()); - } - else if (configuration.withGlobs()) - { - /// Iterate through disclosed globs and make a source for each file - iterator_wrapper = std::make_shared( - object_storage.get(), configuration.container, configuration.blob_path, - query_info.query, virtual_columns, local_context, nullptr, local_context->getFileProgressCallback()); - } - else - { - iterator_wrapper = std::make_shared( - object_storage.get(), configuration.container, configuration.blobs_paths, - query_info.query, virtual_columns, local_context, nullptr, local_context->getFileProgressCallback()); - } + auto this_ptr = std::static_pointer_cast(shared_from_this()); 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; + auto reading = std::make_unique( + read_from_format_info.source_header, + std::move(this_ptr), + std::move(read_from_format_info), + need_only_count, + local_context, + max_block_size, + num_streams); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromAzureBlob::createIterator(const ActionsDAG::Node * predicate) +{ + if (iterator_wrapper) + return; + + const auto & configuration = storage->configuration; + + if (storage->distributed_processing) + { + iterator_wrapper = std::make_shared(context, + context->getReadTaskCallback()); + } + else if (configuration.withGlobs()) + { + /// Iterate through disclosed globs and make a source for each file + iterator_wrapper = std::make_shared( + storage->object_storage.get(), configuration.container, configuration.blob_path, + predicate, storage->virtual_columns, context, nullptr, context->getFileProgressCallback()); + } + else + { + iterator_wrapper = std::make_shared( + storage->object_storage.get(), configuration.container, configuration.blobs_paths, + predicate, storage->virtual_columns, context, nullptr, context->getFileProgressCallback()); + } +} + +void ReadFromAzureBlob::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + createIterator(nullptr); + + const auto & configuration = storage->configuration; + Pipes pipes; + for (size_t i = 0; i < num_streams; ++i) { pipes.emplace_back(std::make_shared( - read_from_format_info, + info, configuration.format, getName(), - local_context, - format_settings, + context, + storage->format_settings, max_block_size, configuration.compression_method, - object_storage.get(), + storage->object_storage.get(), configuration.container, configuration.connection_url, iterator_wrapper, - need_only_count, - query_info)); + need_only_count)); } - return Pipe::unitePipes(std::move(pipes)); + pipeline.init(Pipe::unitePipes(std::move(pipes))); } SinkToStoragePtr StorageAzureBlob::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) @@ -872,6 +950,55 @@ StorageAzureBlobSource::GlobIterator::GlobIterator( recursive = blob_path_with_globs == "/**" ? true : false; } +StorageAzureBlobSource::GlobIterator::GlobIterator( + AzureObjectStorage * object_storage_, + const std::string & container_, + String blob_path_with_globs_, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns_, + ContextPtr context_, + RelativePathsWithMetadata * outer_blobs_, + std::function file_progress_callback_) + : IIterator(context_) + , object_storage(object_storage_) + , container(container_) + , blob_path_with_globs(blob_path_with_globs_) + , virtual_columns(virtual_columns_) + , outer_blobs(outer_blobs_) + , file_progress_callback(file_progress_callback_) +{ + + const String key_prefix = blob_path_with_globs.substr(0, blob_path_with_globs.find_first_of("*?{")); + + /// We don't have to list bucket, because there is no asterisks. + if (key_prefix.size() == blob_path_with_globs.size()) + { + auto object_metadata = object_storage->getObjectMetadata(blob_path_with_globs); + blobs_with_metadata.emplace_back( + blob_path_with_globs, + object_metadata); + if (outer_blobs) + outer_blobs->emplace_back(blobs_with_metadata.back()); + if (file_progress_callback) + file_progress_callback(FileProgress(0, object_metadata.size_bytes)); + is_finished = true; + return; + } + + object_storage_iterator = object_storage->iterate(key_prefix); + + matcher = std::make_unique(makeRegexpPatternFromGlobs(blob_path_with_globs)); + + if (!matcher->ok()) + throw Exception( + ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", blob_path_with_globs, matcher->error()); + + recursive = blob_path_with_globs == "/**" ? true : false; + + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + is_initialized = true; +} + RelativePathWithMetadata StorageAzureBlobSource::GlobIterator::next() { std::lock_guard lock(next_mutex); @@ -924,6 +1051,15 @@ RelativePathWithMetadata StorageAzureBlobSource::GlobIterator::next() VirtualColumnUtils::filterByPathOrFile(new_batch, paths, query, virtual_columns, getContext(), filter_ast); } + else if (filter_dag) + { + std::vector paths; + paths.reserve(new_batch.size()); + for (auto & path_with_metadata : new_batch) + paths.push_back(fs::path(container) / path_with_metadata.relative_path); + + VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_dag, virtual_columns, getContext()); + } if (outer_blobs) outer_blobs->insert(outer_blobs->end(), new_batch.begin(), new_batch.end()); @@ -948,7 +1084,7 @@ StorageAzureBlobSource::KeysIterator::KeysIterator( AzureObjectStorage * object_storage_, const std::string & container_, const Strings & keys_, - ASTPtr query_, + const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns_, ContextPtr context_, RelativePathsWithMetadata * outer_blobs, @@ -956,23 +1092,22 @@ StorageAzureBlobSource::KeysIterator::KeysIterator( : IIterator(context_) , object_storage(object_storage_) , container(container_) - , query(query_) , virtual_columns(virtual_columns_) { Strings all_keys = keys_; ASTPtr filter_ast; if (!all_keys.empty()) - filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, fs::path(container) / all_keys[0], getContext()); + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - if (filter_ast) + if (filter_dag) { Strings paths; paths.reserve(all_keys.size()); for (const auto & key : all_keys) paths.push_back(fs::path(container) / key); - VirtualColumnUtils::filterByPathOrFile(all_keys, paths, query, virtual_columns, getContext(), filter_ast); + VirtualColumnUtils::filterByPathOrFile(all_keys, paths, filter_dag, virtual_columns, getContext()); } for (auto && key : all_keys) @@ -1078,8 +1213,7 @@ StorageAzureBlobSource::StorageAzureBlobSource( const String & container_, const String & connection_url_, std::shared_ptr file_iterator_, - bool need_only_count_, - const SelectQueryInfo & query_info_) + bool need_only_count_) :ISource(info.source_header, false) , WithContext(context_) , requested_columns(info.requested_columns) @@ -1096,7 +1230,6 @@ StorageAzureBlobSource::StorageAzureBlobSource( , connection_url(connection_url_) , file_iterator(file_iterator_) , need_only_count(need_only_count_) - , query_info(query_info_) , create_reader_pool(CurrentMetrics::ObjectStorageAzureThreads, CurrentMetrics::ObjectStorageAzureThreadsActive, CurrentMetrics::ObjectStorageAzureThreadsScheduled, 1) , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "AzureReader")) { diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index bf4f6f37efe..30b91b7f85a 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -88,7 +88,8 @@ public: return name; } - Pipe read( + void read( + QueryPlan & query_plan, const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, @@ -126,6 +127,8 @@ public: bool distributed_processing = false); private: + friend class ReadFromAzureBlob; + std::string name; Configuration configuration; std::unique_ptr object_storage; @@ -162,6 +165,16 @@ public: RelativePathsWithMetadata * outer_blobs_, std::function file_progress_callback_ = {}); + GlobIterator( + AzureObjectStorage * object_storage_, + const std::string & container_, + String blob_path_with_globs_, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns_, + ContextPtr context_, + RelativePathsWithMetadata * outer_blobs_, + std::function file_progress_callback_ = {}); + RelativePathWithMetadata next() override; ~GlobIterator() override = default; @@ -171,6 +184,7 @@ public: String blob_path_with_globs; ASTPtr query; ASTPtr filter_ast; + ActionsDAGPtr filter_dag; NamesAndTypesList virtual_columns; size_t index = 0; @@ -212,7 +226,7 @@ public: AzureObjectStorage * object_storage_, const std::string & container_, const Strings & keys_, - ASTPtr query_, + const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns_, ContextPtr context_, RelativePathsWithMetadata * outer_blobs, @@ -226,7 +240,7 @@ public: std::string container; RelativePathsWithMetadata keys; - ASTPtr query; + ActionsDAGPtr filter_dag; NamesAndTypesList virtual_columns; std::atomic index = 0; @@ -244,8 +258,7 @@ public: const String & container_, const String & connection_url_, std::shared_ptr file_iterator_, - bool need_only_count_, - const SelectQueryInfo & query_info_); + bool need_only_count_); ~StorageAzureBlobSource() override; Chunk generate() override; @@ -271,7 +284,6 @@ private: std::shared_ptr file_iterator; bool need_only_count; size_t total_rows_in_file = 0; - SelectQueryInfo query_info; struct ReaderHolder { From d3d5976d3e93a9fa7f14462ce84a1136e3437fee Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 2 Jan 2024 15:13:25 +0100 Subject: [PATCH 23/61] fix --- src/Interpreters/executeDDLQueryOnCluster.cpp | 3 ++- .../0_stateless/02447_drop_database_replica.reference | 6 ++++-- tests/queries/0_stateless/02447_drop_database_replica.sh | 6 ++++-- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index ba7638cd83f..6b6054fdae3 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -312,7 +312,8 @@ DDLQueryStatusSource::DDLQueryStatusSource( , log(&Poco::Logger::get("DDLQueryStatusSource")) { auto output_mode = context->getSettingsRef().distributed_ddl_output_mode; - throw_on_timeout = output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::NONE; + throw_on_timeout = output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE + || output_mode == DistributedDDLOutputMode::NONE; if (hosts_to_wait) { diff --git a/tests/queries/0_stateless/02447_drop_database_replica.reference b/tests/queries/0_stateless/02447_drop_database_replica.reference index 8ad9008057f..7be5dde1998 100644 --- a/tests/queries/0_stateless/02447_drop_database_replica.reference +++ b/tests/queries/0_stateless/02447_drop_database_replica.reference @@ -13,11 +13,12 @@ t rdb_default 1 1 s1 r1 1 2 s1 r1 OK 2 0 -s2 r1 QUEUED 2 0 s1 r2 QUEUED 2 0 +s2 r1 QUEUED 2 0 +2 s1 r1 OK 2 0 -s2 r1 QUEUED 2 0 s1 r2 QUEUED 2 0 +s2 r1 QUEUED 2 0 2 rdb_default 1 1 s1 r1 1 rdb_default 1 2 s1 r2 0 @@ -26,4 +27,5 @@ rdb_default 1 2 s1 r2 0 t t2 t3 +t4 rdb_default_4 1 1 s1 r1 1 diff --git a/tests/queries/0_stateless/02447_drop_database_replica.sh b/tests/queries/0_stateless/02447_drop_database_replica.sh index 388af3fad74..d12f173f388 100755 --- a/tests/queries/0_stateless/02447_drop_database_replica.sh +++ b/tests/queries/0_stateless/02447_drop_database_replica.sh @@ -32,8 +32,10 @@ $CLICKHOUSE_CLIENT -q "system sync database replica $db" $CLICKHOUSE_CLIENT -q "select cluster, shard_num, replica_num, database_shard_name, database_replica_name, is_active from system.clusters where cluster='$db' and shard_num=1 and replica_num=1" $CLICKHOUSE_CLIENT -q "system drop database replica 's1|r1' from database $db2" 2>&1| grep -Fac "is active, cannot drop it" -$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=throw_only_active -q "create table $db.t2 (n int) engine=Log" -$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=null_status_on_timeout_only_active -q "create table $db.t3 (n int) engine=Log" +# Also check that it doesn't exceed distributed_ddl_task_timeout waiting for inactive replicas +timeout 10s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=throw_only_active -q "create table $db.t2 (n int) engine=Log" 2>/dev/null | sort +timeout 10s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=throw_only_active -q "create table $db.t3 (n int) engine=Log" 2>&1| grep -Fac "TIMEOUT_EXCEEDED" +timeout 10s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=null_status_on_timeout_only_active -q "create table $db.t4 (n int) engine=Log" | sort $CLICKHOUSE_CLIENT -q "detach database $db3" $CLICKHOUSE_CLIENT -q "system drop database replica 'r1' from shard 's2' from database $db" From 3e3fed1cbe2b6b67c02a852164653a8b241c672a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 2 Jan 2024 15:18:13 +0000 Subject: [PATCH 24/61] Add reading step to URL --- src/Storages/HDFS/StorageHDFS.cpp | 14 +- src/Storages/S3Queue/StorageS3Queue.cpp | 19 +- src/Storages/StorageAzureBlob.cpp | 10 +- src/Storages/StorageFile.cpp | 5 + src/Storages/StorageS3.cpp | 10 +- src/Storages/StorageURL.cpp | 285 +++++++++++++++++++----- src/Storages/StorageURL.h | 10 +- src/Storages/StorageXDBC.cpp | 5 +- src/Storages/StorageXDBC.h | 3 +- 9 files changed, 280 insertions(+), 81 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index fe37b2eb57a..c7cbaa1e561 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -1,3 +1,4 @@ +#include "Processors/Sources/NullSource.h" #include "config.h" #if USE_HDFS @@ -1014,10 +1015,17 @@ void ReadFromHDFS::initializePipeline(QueryPipelineBuilder & pipeline, const Bui context, max_block_size, iterator_wrapper, - need_only_count)); //, - //query_info)); + need_only_count)); } - pipeline.init(Pipe::unitePipes(std::move(pipes))); + + auto pipe = Pipe::unitePipes(std::move(pipes)); + if (pipe.empty()) + pipe = Pipe(std::make_shared(info.source_header)); + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); } SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context_, bool /*async_insert*/) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 1a6666c00d0..6d078e1aa1b 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -1,5 +1,6 @@ #include "Processors/QueryPlan/QueryPlan.h" #include "Processors/QueryPlan/SourceStepWithFilter.h" +#include "Processors/Sources/NullSource.h" #include "QueryPipeline/QueryPipelineBuilder.h" #include "config.h" @@ -218,16 +219,12 @@ public: Block sample_block, ReadFromFormatInfo info_, std::shared_ptr storage_, - // StorageSnapshotPtr storage_snapshot_, - // Names column_names_, ContextPtr context_, size_t max_block_size_, size_t num_streams_) : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) , info(std::move(info_)) , storage(std::move(storage_)) - // , storage_snapshot(std::move(storage_snapshot_)) - // , column_names(std::move(column_names_)) , context(std::move(context_)) , max_block_size(max_block_size_) , num_streams(num_streams_) @@ -237,8 +234,6 @@ public: private: ReadFromFormatInfo info; std::shared_ptr storage; - // StorageSnapshotPtr storage_snapshot; - // Names column_names; ContextPtr context; size_t max_block_size; size_t num_streams; @@ -296,8 +291,6 @@ void StorageS3Queue::read( read_from_format_info.source_header, read_from_format_info, std::move(this_ptr), - // storage_snapshot, - // column_names, local_context, max_block_size, num_streams); @@ -313,7 +306,15 @@ void ReadFromS3Queue::initializePipeline(QueryPipelineBuilder & pipeline, const createIterator(nullptr); for (size_t i = 0; i < adjusted_num_streams; ++i) pipes.emplace_back(storage->createSource(info, iterator, max_block_size, context)); - pipeline.init(Pipe::unitePipes(std::move(pipes))); + + auto pipe = Pipe::unitePipes(std::move(pipes)); + if (pipe.empty()) + pipe = Pipe(std::make_shared(info.source_header)); + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); } std::shared_ptr StorageS3Queue::createSource( diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 048248ef334..defff830411 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1,6 +1,7 @@ #include #include "Processors/QueryPlan/QueryPlan.h" #include "Processors/QueryPlan/SourceStepWithFilter.h" +#include "Processors/Sources/NullSource.h" #if USE_AZURE_BLOB_STORAGE @@ -800,7 +801,14 @@ void ReadFromAzureBlob::initializePipeline(QueryPipelineBuilder & pipeline, cons need_only_count)); } - pipeline.init(Pipe::unitePipes(std::move(pipes))); + auto pipe = Pipe::unitePipes(std::move(pipes)); + if (pipe.empty()) + pipe = Pipe(std::make_shared(info.source_header)); + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); } SinkToStoragePtr StorageAzureBlob::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 12a8eed106e..18acbfc7153 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1477,6 +1477,8 @@ void ReadFromFile::createIterator(const ActionsDAG::Node * predicate) void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { + createIterator(nullptr); + size_t num_streams = max_num_streams; size_t files_to_read = 0; @@ -1523,6 +1525,9 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui if (parallelize_output && storage->parallelizeOutputAfterReading(context) && output_ports > 0 && output_ports < max_num_streams) pipe.resize(max_num_streams); + if (pipe.empty()) + pipe = Pipe(std::make_shared(info.source_header)); + for (const auto & processor : pipe.getProcessors()) processors.emplace_back(processor); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 780a2755bcf..375a367bfab 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1,3 +1,4 @@ +#include "Processors/Sources/NullSource.h" #include "config.h" #include #include "Parsers/ASTCreateQuery.h" @@ -1352,7 +1353,14 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, need_only_count)); } - pipeline.init(Pipe::unitePipes(std::move(pipes))); + auto pipe = Pipe::unitePipes(std::move(pipes)); + if (pipe.empty()) + pipe = Pipe(std::make_shared(read_from_format_info.source_header)); + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); } SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index d6b6f5af61c..3f88966e3d3 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -34,6 +34,8 @@ #include #include #include +#include "Processors/QueryPlan/QueryPlan.h" +#include "Processors/QueryPlan/SourceStepWithFilter.h" #include #include @@ -201,6 +203,25 @@ public: } } + Impl(const String & uri_, size_t max_addresses, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) + { + uris = parseRemoteDescription(uri_, 0, uri_.size(), ',', max_addresses); + + ActionsDAGPtr filter_dag; + if (!uris.empty()) + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + + if (filter_dag) + { + std::vector paths; + paths.reserve(uris.size()); + for (const auto & uri : uris) + paths.push_back(Poco::URI(uri).getPath()); + + VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, context); + } + } + String next() { size_t current_index = index.fetch_add(1, std::memory_order_relaxed); @@ -223,6 +244,9 @@ private: StorageURLSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, size_t max_addresses, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context) : pimpl(std::make_shared(uri, max_addresses, query, virtual_columns, context)) {} +StorageURLSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, size_t max_addresses, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) + : pimpl(std::make_shared(uri, max_addresses, predicate, virtual_columns, context)) {} + String StorageURLSource::DisclosedGlobIterator::next() { return pimpl->next(); @@ -260,7 +284,6 @@ StorageURLSource::StorageURLSource( const ConnectionTimeouts & timeouts, CompressionMethod compression_method, size_t max_parsing_threads, - const SelectQueryInfo &, const HTTPHeaderEntries & headers_, const URIParams & params, bool glob_url, @@ -874,7 +897,86 @@ bool IStorageURLBase::parallelizeOutputAfterReading(ContextPtr context) const return FormatFactory::instance().checkParallelizeOutputAfterReading(format_name, context); } -Pipe IStorageURLBase::read( +class ReadFromURL : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromURL"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + void applyFilters() override; + + ReadFromURL( + Block sample_block, + std::shared_ptr storage_, + std::vector * uri_options_, + ReadFromFormatInfo info_, + const bool need_only_count_, + std::vector> read_uri_params_, + std::function read_post_data_callback_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_) + : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) + , storage(std::move(storage_)) + , uri_options(uri_options_) + // , paths(std::move(paths_)) + // , archive_info(std::move(archive_info_)) + // , virtual_columns(std::move(virtual_columns_)) + // , distributed_processing(distributed_processing_) + , info(std::move(info_)) + , need_only_count(need_only_count_) + , read_uri_params(std::move(read_uri_params_)) + , read_post_data_callback(std::move(read_post_data_callback_)) + // , total_bytes_to_read(total_bytes_to_read_) + , context(std::move(context_)) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + { + } + +private: + std::shared_ptr storage; + std::vector * uri_options; + + // std::vector paths; + // std::optional archive_info; + + // NamesAndTypesList virtual_columns; + // const bool distributed_processing; + + ReadFromFormatInfo info; + const bool need_only_count; + std::vector> read_uri_params; + std::function read_post_data_callback; + + // size_t total_bytes_to_read; + + ContextPtr context; + + size_t max_block_size; + size_t num_streams; + + std::shared_ptr iterator_wrapper; + bool is_url_with_globs = false; + bool is_empty_glob = false; + + // FieldVectorPtr keys; + // bool all_scan = false; + + void createIterator(const ActionsDAG::Node * predicate); +}; + +void ReadFromURL::applyFilters() +{ + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + createIterator(predicate); +} + +void IStorageURLBase::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -884,16 +986,61 @@ Pipe IStorageURLBase::read( size_t num_streams) { auto params = getReadURIParams(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size); - - std::shared_ptr iterator_wrapper{nullptr}; - bool is_url_with_globs = urlWithGlobs(uri); - size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements; auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); - if (distributed_processing) + bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) + && local_context->getSettingsRef().optimize_count_from_files; + + auto read_post_data_callback = getReadPOSTDataCallback( + read_from_format_info.columns_description.getNamesOfPhysical(), + read_from_format_info.columns_description, + query_info, + local_context, + processed_stage, + max_block_size); + + auto this_ptr = std::static_pointer_cast(shared_from_this()); + + auto reading = std::make_unique( + read_from_format_info.source_header, + std::move(this_ptr), + nullptr, + std::move(read_from_format_info), + need_only_count, + std::move(params), + std::move(read_post_data_callback), + local_context, + max_block_size, + num_streams); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromURL::createIterator(const ActionsDAG::Node * predicate) +{ + if (iterator_wrapper || is_empty_glob) + return; + + if (uri_options) + { + iterator_wrapper = std::make_shared([&, done = false]() mutable + { + if (done) + return StorageURLSource::FailoverOptions{}; + done = true; + return *uri_options; + }); + + return; + } + + size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; + is_url_with_globs = urlWithGlobs(storage->uri); + + if (storage->distributed_processing) { iterator_wrapper = std::make_shared( - [callback = local_context->getReadTaskCallback(), max_addresses]() + [callback = context->getReadTaskCallback(), max_addresses]() { String next_uri = callback(); if (next_uri.empty()) @@ -904,11 +1051,14 @@ Pipe IStorageURLBase::read( else if (is_url_with_globs) { /// Iterate through disclosed globs and make a source for each file - auto glob_iterator = std::make_shared(uri, max_addresses, query_info.query, virtual_columns, local_context); + auto glob_iterator = std::make_shared(storage->uri, max_addresses, predicate, storage->virtual_columns, context); /// check if we filtered out all the paths if (glob_iterator->size() == 0) - return Pipe(std::make_shared(read_from_format_info.source_header)); + { + is_empty_glob = true; + return; + } iterator_wrapper = std::make_shared([glob_iterator, max_addresses]() { @@ -928,53 +1078,70 @@ Pipe IStorageURLBase::read( if (done) return StorageURLSource::FailoverOptions{}; done = true; - return getFailoverOptions(uri, max_addresses); + return getFailoverOptions(storage->uri, max_addresses); }); num_streams = 1; } +} - bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) - && local_context->getSettingsRef().optimize_count_from_files; +void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + createIterator(nullptr); + + if (is_empty_glob) + { + pipeline.init(Pipe(std::make_shared(info.source_header))); + return; + } Pipes pipes; pipes.reserve(num_streams); - const size_t max_threads = local_context->getSettingsRef().max_threads; + const size_t max_threads = context->getSettingsRef().max_threads; const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / num_streams); for (size_t i = 0; i < num_streams; ++i) { pipes.emplace_back(std::make_shared( - read_from_format_info, + info, iterator_wrapper, - getReadMethod(), - getReadPOSTDataCallback( - read_from_format_info.columns_description.getNamesOfPhysical(), - read_from_format_info.columns_description, - query_info, - local_context, - processed_stage, - max_block_size), - format_name, - format_settings, - getName(), - local_context, + storage->getReadMethod(), + read_post_data_callback, + storage->format_name, + storage->format_settings, + storage->getName(), + context, max_block_size, - getHTTPTimeouts(local_context), - compression_method, + getHTTPTimeouts(context), + storage->compression_method, max_parsing_threads, - query_info, - headers, - params, + storage->headers, + read_uri_params, is_url_with_globs, need_only_count)); } - return Pipe::unitePipes(std::move(pipes)); + if (uri_options) + std::shuffle(uri_options->begin(), uri_options->end(), thread_local_rng); + + auto pipe = Pipe::unitePipes(std::move(pipes)); + size_t output_ports = pipe.numOutputPorts(); + const bool parallelize_output = context->getSettingsRef().parallelize_output_from_storages; + if (parallelize_output && storage->parallelizeOutputAfterReading(context) && output_ports > 0 && output_ports < num_streams) + pipe.resize(num_streams); + + if (pipe.empty()) + pipe = Pipe(std::make_shared(info.source_header)); + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); } -Pipe StorageURLWithFailover::read( +void StorageURLWithFailover::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -984,38 +1151,34 @@ Pipe StorageURLWithFailover::read( size_t num_streams) { auto params = getReadURIParams(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size); - - auto iterator_wrapper = std::make_shared([&, done = false]() mutable - { - if (done) - return StorageURLSource::FailoverOptions{}; - done = true; - return uri_options; - }); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); - 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); + bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) + && local_context->getSettingsRef().optimize_count_from_files; - auto pipe = Pipe(std::make_shared( - read_from_format_info, - iterator_wrapper, - getReadMethod(), - getReadPOSTDataCallback(read_from_format_info.columns_description.getNamesOfPhysical(), read_from_format_info.columns_description, query_info, local_context, processed_stage, max_block_size), - format_name, - format_settings, - getName(), + auto read_post_data_callback = getReadPOSTDataCallback( + read_from_format_info.columns_description.getNamesOfPhysical(), + read_from_format_info.columns_description, + query_info, + local_context, + processed_stage, + max_block_size); + + auto this_ptr = std::static_pointer_cast(shared_from_this()); + + auto reading = std::make_unique( + read_from_format_info.source_header, + std::move(this_ptr), + &uri_options, + std::move(read_from_format_info), + need_only_count, + std::move(params), + std::move(read_post_data_callback), local_context, max_block_size, - getHTTPTimeouts(local_context), - compression_method, - max_parsing_threads, - query_info, - headers, - params)); - std::shuffle(uri_options.begin(), uri_options.end(), thread_local_rng); - return pipe; + num_streams); + + query_plan.addStep(std::move(reading)); } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 8d027025882..1b2fb97cb28 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -34,7 +34,8 @@ class PullingPipelineExecutor; class IStorageURLBase : public IStorage { public: - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -67,6 +68,8 @@ public: const ContextPtr & context); protected: + friend class ReadFromURL; + IStorageURLBase( const String & uri_, ContextPtr context_, @@ -137,6 +140,7 @@ public: { public: DisclosedGlobIterator(const String & uri_, size_t max_addresses, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context); + DisclosedGlobIterator(const String & uri_, size_t max_addresses, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context); String next(); size_t size(); @@ -162,7 +166,6 @@ public: const ConnectionTimeouts & timeouts, CompressionMethod compression_method, size_t max_parsing_threads, - const SelectQueryInfo & query_info, const HTTPHeaderEntries & headers_ = {}, const URIParams & params = {}, bool glob_url = false, @@ -317,7 +320,8 @@ public: ContextPtr context_, const String & compression_method_); - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index a569c50835c..a274b1ba4db 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -102,7 +102,8 @@ std::function StorageXDBC::getReadPOSTDataCallback( return write_body_callback; } -Pipe StorageXDBC::read( +void StorageXDBC::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -114,7 +115,7 @@ Pipe StorageXDBC::read( storage_snapshot->check(column_names); bridge_helper->startBridgeSync(); - return IStorageURLBase::read(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); + IStorageURLBase::read(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); } SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index 1c1651cb333..fe678785dc2 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -19,7 +19,8 @@ namespace DB class StorageXDBC : public IStorageURLBase { public: - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, From f2dfe8bddabb05194d0c380df13e8ae836fc24fa Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 2 Jan 2024 16:42:17 +0100 Subject: [PATCH 25/61] Fix build --- src/Storages/StorageMaterializedView.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 458e0c9ab6b..59f1d5eee1b 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -72,7 +72,7 @@ public: StoragePtr getTargetTable() const; StoragePtr tryGetTargetTable() const; - const StorageID & getTargetTableId() const { return target_table_id; } + StorageID getTargetTableId() const { return target_table_id; } /// Get the virtual column of the target table; NamesAndTypesList getVirtuals() const override; @@ -119,7 +119,6 @@ private: std::tuple> prepareRefresh() const; StorageID exchangeTargetTable(StorageID fresh_table, ContextPtr refresh_context); - StorageID getTargetTableId() const; void setTargetTableId(StorageID id); void updateTargetTableId(std::optional database_name, std::optional table_name); }; From 8936c8376a05030b5559364cd65ef4db5ab7af87 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 2 Jan 2024 17:14:16 +0000 Subject: [PATCH 26/61] Use predicate in getTaskIteratorExtension. --- src/Storages/HDFS/StorageHDFSCluster.cpp | 4 +- src/Storages/HDFS/StorageHDFSCluster.h | 2 +- src/Storages/IStorageCluster.cpp | 110 ++++++++++++++++++++--- src/Storages/IStorageCluster.h | 7 +- src/Storages/StorageAzureBlobCluster.cpp | 4 +- src/Storages/StorageAzureBlobCluster.h | 2 +- src/Storages/StorageDistributed.cpp | 61 ++++++++++++- src/Storages/StorageFileCluster.cpp | 4 +- src/Storages/StorageFileCluster.h | 2 +- src/Storages/StorageS3Cluster.cpp | 4 +- src/Storages/StorageS3Cluster.h | 2 +- src/Storages/StorageURL.cpp | 6 +- src/Storages/StorageURLCluster.cpp | 4 +- src/Storages/StorageURLCluster.h | 2 +- 14 files changed, 174 insertions(+), 40 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index bff22936e95..2e8129b9845 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -79,9 +79,9 @@ void StorageHDFSCluster::addColumnsStructureToQuery(ASTPtr & query, const String } -RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const +RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const { - auto iterator = std::make_shared(uri, query, virtual_columns, context); + auto iterator = std::make_shared(uri, predicate, virtual_columns, context); auto callback = std::make_shared>([iter = std::move(iterator)]() mutable -> String { return iter->next().path; }); return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)}; } diff --git a/src/Storages/HDFS/StorageHDFSCluster.h b/src/Storages/HDFS/StorageHDFSCluster.h index 8ad4a83c5b9..7c4c41a573a 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.h +++ b/src/Storages/HDFS/StorageHDFSCluster.h @@ -35,7 +35,7 @@ public: NamesAndTypesList getVirtuals() const override; - RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override; + RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override; bool supportsSubcolumns() const override { return true; } diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp index 1447dad1374..c59b74255b2 100644 --- a/src/Storages/IStorageCluster.cpp +++ b/src/Storages/IStorageCluster.cpp @@ -2,6 +2,9 @@ #include "Common/Exception.h" #include "Core/QueryProcessingStage.h" +#include "Processors/QueryPlan/SourceStepWithFilter.h" +#include "Processors/Sources/NullSource.h" +#include "QueryPipeline/QueryPipelineBuilder.h" #include #include #include @@ -38,9 +41,66 @@ IStorageCluster::IStorageCluster( { } +class ReadFromCluster : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromCluster"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + void applyFilters() override; + + ReadFromCluster( + Block sample_block, + std::shared_ptr storage_, + ASTPtr query_to_send_, + QueryProcessingStage::Enum processed_stage_, + ClusterPtr cluster_, + Poco::Logger * log_, + ContextPtr context_) + : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) + , storage(std::move(storage_)) + , query_to_send(std::move(query_to_send_)) + , processed_stage(processed_stage_) + , cluster(std::move(cluster_)) + , log(log_) + , context(std::move(context_)) + { + } + +private: + std::shared_ptr storage; + ASTPtr query_to_send; + QueryProcessingStage::Enum processed_stage; + ClusterPtr cluster; + Poco::Logger * log; + ContextPtr context; + + std::optional extension; + + void createExtension(const ActionsDAG::Node * predicate); + ContextPtr updateSettings(const Settings & settings); +}; + +void ReadFromCluster::applyFilters() +{ + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context); + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + createExtension(predicate); +} + +void ReadFromCluster::createExtension(const ActionsDAG::Node * predicate) +{ + if (extension) + return; + + extension = storage->getTaskIteratorExtension(predicate, context); +} /// The code executes on initiator -Pipe IStorageCluster::read( +void IStorageCluster::read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -49,10 +109,10 @@ Pipe IStorageCluster::read( size_t /*max_block_size*/, size_t /*num_streams*/) { - updateBeforeRead(context); + storage_snapshot->check(column_names); + updateBeforeRead(context); auto cluster = getCluster(context); - auto extension = getTaskIteratorExtension(query_info.query, context); /// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*) @@ -70,12 +130,6 @@ Pipe IStorageCluster::read( query_to_send = interpreter.getQueryInfo().query->clone(); } - const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; - - Pipes pipes; - - const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; - if (!structure_argument_was_provided) addColumnsStructureToQuery(query_to_send, storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), context); @@ -89,7 +143,29 @@ Pipe IStorageCluster::read( /* only_replace_in_join_= */true); visitor.visit(query_to_send); - auto new_context = updateSettings(context, context->getSettingsRef()); + auto this_ptr = std::static_pointer_cast(shared_from_this()); + + auto reading = std::make_unique( + sample_block, + std::move(this_ptr), + std::move(query_to_send), + processed_stage, + cluster, + log, + context); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromCluster::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + createExtension(nullptr); + + const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; + const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; + + Pipes pipes; + auto new_context = updateSettings(context->getSettingsRef()); const auto & current_settings = new_context->getSettingsRef(); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); for (const auto & shard_info : cluster->getShardsInfo()) @@ -100,7 +176,7 @@ Pipe IStorageCluster::read( auto remote_query_executor = std::make_shared( std::vector{try_result}, queryToString(query_to_send), - sample_block, + getOutputStream().header, new_context, /*throttler=*/nullptr, scalars, @@ -113,8 +189,14 @@ Pipe IStorageCluster::read( } } - storage_snapshot->check(column_names); - return Pipe::unitePipes(std::move(pipes)); + auto pipe = Pipe::unitePipes(std::move(pipes)); + if (pipe.empty()) + pipe = Pipe(std::make_shared(getOutputStream().header)); + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); } QueryProcessingStage::Enum IStorageCluster::getQueryProcessingStage( @@ -129,7 +211,7 @@ QueryProcessingStage::Enum IStorageCluster::getQueryProcessingStage( return QueryProcessingStage::Enum::FetchColumns; } -ContextPtr IStorageCluster::updateSettings(ContextPtr context, const Settings & settings) +ContextPtr ReadFromCluster::updateSettings(const Settings & settings) { Settings new_settings = settings; diff --git a/src/Storages/IStorageCluster.h b/src/Storages/IStorageCluster.h index b15ed37202a..b233f20103d 100644 --- a/src/Storages/IStorageCluster.h +++ b/src/Storages/IStorageCluster.h @@ -22,7 +22,8 @@ public: Poco::Logger * log_, bool structure_argument_was_provided_); - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -33,7 +34,7 @@ public: ClusterPtr getCluster(ContextPtr context) const; /// Query is needed for pruning by virtual columns (_file, _path) - virtual RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const = 0; + virtual RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const = 0; QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; @@ -45,8 +46,6 @@ protected: virtual void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) = 0; private: - ContextPtr updateSettings(ContextPtr context, const Settings & settings); - Poco::Logger * log; String cluster_name; bool structure_argument_was_provided; diff --git a/src/Storages/StorageAzureBlobCluster.cpp b/src/Storages/StorageAzureBlobCluster.cpp index b8f95458379..a6372577fb0 100644 --- a/src/Storages/StorageAzureBlobCluster.cpp +++ b/src/Storages/StorageAzureBlobCluster.cpp @@ -69,11 +69,11 @@ void StorageAzureBlobCluster::addColumnsStructureToQuery(ASTPtr & query, const S TableFunctionAzureBlobStorageCluster::addColumnsStructureToArguments(expression_list->children, structure, context); } -RemoteQueryExecutor::Extension StorageAzureBlobCluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const +RemoteQueryExecutor::Extension StorageAzureBlobCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const { auto iterator = std::make_shared( object_storage.get(), configuration.container, configuration.blob_path, - query, virtual_columns, context, nullptr); + predicate, virtual_columns, context, nullptr); auto callback = std::make_shared>([iterator]() mutable -> String{ return iterator->next().relative_path; }); return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; } diff --git a/src/Storages/StorageAzureBlobCluster.h b/src/Storages/StorageAzureBlobCluster.h index 2900243708c..2831b94f825 100644 --- a/src/Storages/StorageAzureBlobCluster.h +++ b/src/Storages/StorageAzureBlobCluster.h @@ -34,7 +34,7 @@ public: NamesAndTypesList getVirtuals() const override; - RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override; + RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override; bool supportsSubcolumns() const override { return true; } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index a928a4daf63..c914388e55e 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -30,6 +30,7 @@ #include #include #include +#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" #include #include @@ -1068,15 +1069,67 @@ std::optional StorageDistributed::distributedWriteBetweenDistribu return pipeline; } +static ActionsDAGPtr getFilterFromQuery(const ASTPtr & ast, ContextPtr context) +{ + QueryPlan plan; + SelectQueryOptions options; + options.only_analyze = true; + if (context->getSettingsRef().allow_experimental_analyzer) + { + InterpreterSelectQueryAnalyzer interpreter(ast, context, options); + plan = std::move(interpreter).extractQueryPlan(); + } + else + { + InterpreterSelectWithUnionQuery interpreter(ast, context, options); + interpreter.buildQueryPlan(plan); + } + + plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); + + std::stack nodes; + nodes.push(plan.getRootNode()); + + SourceStepWithFilter * source = nullptr; + + while (!nodes.empty()) + { + const auto * node = nodes.top(); + nodes.pop(); + + if (auto * with_filter = dynamic_cast(node->step.get())) + { + if (source) + { + WriteBufferFromOwnString buf; + plan.explainPlan(buf, {}); + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Found multiple source steps for query\n{}\nPlan\n{}", + queryToString(ast), buf.str()); + } + + source = with_filter; + } + } + + if (!source) + return nullptr; + + return ActionsDAG::buildFilterActionsDAG(source->getFilterNodes().nodes, {}, context); +} + std::optional StorageDistributed::distributedWriteFromClusterStorage(const IStorageCluster & src_storage_cluster, const ASTInsertQuery & query, ContextPtr local_context) const { const auto & settings = local_context->getSettingsRef(); - auto & select = query.select->as(); + + auto filter = getFilterFromQuery(query.select, local_context); + const ActionsDAG::Node * predicate = nullptr; + if (filter) + predicate = filter->getOutputs().at(0); + /// Select query is needed for pruining on virtual columns - auto extension = src_storage_cluster.getTaskIteratorExtension( - select.list_of_selects->children.at(0)->as()->clone(), - local_context); + auto extension = src_storage_cluster.getTaskIteratorExtension(predicate, local_context); auto dst_cluster = getCluster(); diff --git a/src/Storages/StorageFileCluster.cpp b/src/Storages/StorageFileCluster.cpp index 782c36c9819..c12124f1e07 100644 --- a/src/Storages/StorageFileCluster.cpp +++ b/src/Storages/StorageFileCluster.cpp @@ -71,9 +71,9 @@ void StorageFileCluster::addColumnsStructureToQuery(ASTPtr & query, const String TableFunctionFileCluster::addColumnsStructureToArguments(expression_list->children, structure, context); } -RemoteQueryExecutor::Extension StorageFileCluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const +RemoteQueryExecutor::Extension StorageFileCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const { - auto iterator = std::make_shared(paths, std::nullopt, query, virtual_columns, context); + auto iterator = std::make_shared(paths, std::nullopt, predicate, virtual_columns, context); auto callback = std::make_shared([iter = std::move(iterator)]() mutable -> String { return iter->next(); }); return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)}; } diff --git a/src/Storages/StorageFileCluster.h b/src/Storages/StorageFileCluster.h index e907fbad0de..a6e57c3bb4f 100644 --- a/src/Storages/StorageFileCluster.h +++ b/src/Storages/StorageFileCluster.h @@ -31,7 +31,7 @@ public: NamesAndTypesList getVirtuals() const override { return virtual_columns; } - RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override; + RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override; bool supportsSubcolumns() const override { return true; } diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 702b1f14ae7..e1738056e9d 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -78,10 +78,10 @@ void StorageS3Cluster::updateConfigurationIfChanged(ContextPtr local_context) s3_configuration.update(local_context); } -RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const +RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const { auto iterator = std::make_shared( - *s3_configuration.client, s3_configuration.url, query, virtual_columns, context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback()); + *s3_configuration.client, s3_configuration.url, predicate, virtual_columns, context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback()); auto callback = std::make_shared>([iterator]() mutable -> String { diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h index 81fb48d2398..c526f14834a 100644 --- a/src/Storages/StorageS3Cluster.h +++ b/src/Storages/StorageS3Cluster.h @@ -34,7 +34,7 @@ public: NamesAndTypesList getVirtuals() const override; - RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override; + RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override; bool supportsSubcolumns() const override { return true; } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 3f88966e3d3..ac17f880738 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -906,7 +906,7 @@ public: ReadFromURL( Block sample_block, - std::shared_ptr storage_, + std::shared_ptr storage_, std::vector * uri_options_, ReadFromFormatInfo info_, const bool need_only_count_, @@ -934,7 +934,7 @@ public: } private: - std::shared_ptr storage; + std::shared_ptr storage; std::vector * uri_options; // std::vector paths; @@ -999,7 +999,7 @@ void IStorageURLBase::read( processed_stage, max_block_size); - auto this_ptr = std::static_pointer_cast(shared_from_this()); + auto this_ptr = std::static_pointer_cast(shared_from_this()); auto reading = std::make_unique( read_from_format_info.source_header, diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index c052e781877..a0b5fcd6f28 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -81,9 +81,9 @@ void StorageURLCluster::addColumnsStructureToQuery(ASTPtr & query, const String TableFunctionURLCluster::addColumnsStructureToArguments(expression_list->children, structure, context); } -RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const +RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const { - auto iterator = std::make_shared(uri, context->getSettingsRef().glob_expansion_max_elements, query, virtual_columns, context); + auto iterator = std::make_shared(uri, context->getSettingsRef().glob_expansion_max_elements, predicate, virtual_columns, context); auto callback = std::make_shared([iter = std::move(iterator)]() mutable -> String { return iter->next(); }); return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)}; } diff --git a/src/Storages/StorageURLCluster.h b/src/Storages/StorageURLCluster.h index ddf7e6f0790..07978040029 100644 --- a/src/Storages/StorageURLCluster.h +++ b/src/Storages/StorageURLCluster.h @@ -34,7 +34,7 @@ public: NamesAndTypesList getVirtuals() const override { return virtual_columns; } - RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override; + RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override; bool supportsSubcolumns() const override { return true; } From c808b03e55882beaff7e9e58208546af9cd34760 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 2 Jan 2024 17:27:33 +0000 Subject: [PATCH 27/61] Remove unneeded code --- src/Storages/HDFS/StorageHDFS.cpp | 33 ------------- src/Storages/HDFS/StorageHDFS.h | 1 - src/Storages/StorageAzureBlob.cpp | 64 +----------------------- src/Storages/StorageAzureBlob.h | 13 ----- src/Storages/StorageFile.cpp | 18 ------- src/Storages/StorageFile.h | 8 --- src/Storages/StorageS3.cpp | 82 +------------------------------ src/Storages/StorageS3.h | 10 ---- src/Storages/StorageURL.cpp | 22 --------- src/Storages/StorageURL.h | 1 - 10 files changed, 2 insertions(+), 250 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index c7cbaa1e561..430ecc7a585 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -440,34 +440,6 @@ public: uris_iter = uris.begin(); } - Impl(const String & uri, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - { - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); - uris = getPathsList(path_from_uri, uri_without_path, context); - ASTPtr filter_ast; - if (!uris.empty()) - filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, uris[0].path, context); - - if (filter_ast) - { - std::vector paths; - paths.reserve(uris.size()); - for (const auto & path_with_info : uris) - paths.push_back(path_with_info.path); - - VirtualColumnUtils::filterByPathOrFile(uris, paths, query, virtual_columns, context, filter_ast); - } - auto file_progress_callback = context->getFileProgressCallback(); - - for (auto & elem : uris) - { - elem.path = uri_without_path + elem.path; - if (file_progress_callback && elem.info) - file_progress_callback(FileProgress(0, elem.info->size)); - } - uris_iter = uris.begin(); - } - StorageHDFS::PathWithInfo next() { std::lock_guard lock(mutex); @@ -549,9 +521,6 @@ private: std::function file_progress_callback; }; -HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - : pimpl(std::make_shared(uri, query, virtual_columns, context)) {} - HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) : pimpl(std::make_shared(uri, predicate, virtual_columns, context)) {} @@ -577,7 +546,6 @@ HDFSSource::HDFSSource( UInt64 max_block_size_, std::shared_ptr file_iterator_, bool need_only_count_) - //const SelectQueryInfo & query_info_) : ISource(info.source_header, false) , WithContext(context_) , storage(std::move(storage_)) @@ -588,7 +556,6 @@ HDFSSource::HDFSSource( , file_iterator(file_iterator_) , columns_description(info.columns_description) , need_only_count(need_only_count_) - //, query_info(query_info_) { initialize(); } diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index cee1b674eb7..9d9a857bf4e 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -115,7 +115,6 @@ public: class DisclosedGlobIterator { public: - DisclosedGlobIterator(const String & uri_, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context); DisclosedGlobIterator(const String & uri_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context); StorageHDFS::PathWithInfo next(); private: diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index defff830411..294a65c067f 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -911,53 +911,6 @@ bool StorageAzureBlob::parallelizeOutputAfterReading(ContextPtr context) const return FormatFactory::instance().checkParallelizeOutputAfterReading(configuration.format, context); } -StorageAzureBlobSource::GlobIterator::GlobIterator( - AzureObjectStorage * object_storage_, - const std::string & container_, - String blob_path_with_globs_, - ASTPtr query_, - const NamesAndTypesList & virtual_columns_, - ContextPtr context_, - RelativePathsWithMetadata * outer_blobs_, - std::function file_progress_callback_) - : IIterator(context_) - , object_storage(object_storage_) - , container(container_) - , blob_path_with_globs(blob_path_with_globs_) - , query(query_) - , virtual_columns(virtual_columns_) - , outer_blobs(outer_blobs_) - , file_progress_callback(file_progress_callback_) -{ - - const String key_prefix = blob_path_with_globs.substr(0, blob_path_with_globs.find_first_of("*?{")); - - /// We don't have to list bucket, because there is no asterisks. - if (key_prefix.size() == blob_path_with_globs.size()) - { - auto object_metadata = object_storage->getObjectMetadata(blob_path_with_globs); - blobs_with_metadata.emplace_back( - blob_path_with_globs, - object_metadata); - if (outer_blobs) - outer_blobs->emplace_back(blobs_with_metadata.back()); - if (file_progress_callback) - file_progress_callback(FileProgress(0, object_metadata.size_bytes)); - is_finished = true; - return; - } - - object_storage_iterator = object_storage->iterate(key_prefix); - - matcher = std::make_unique(makeRegexpPatternFromGlobs(blob_path_with_globs)); - - if (!matcher->ok()) - throw Exception( - ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", blob_path_with_globs, matcher->error()); - - recursive = blob_path_with_globs == "/**" ? true : false; -} - StorageAzureBlobSource::GlobIterator::GlobIterator( AzureObjectStorage * object_storage_, const std::string & container_, @@ -1004,7 +957,6 @@ StorageAzureBlobSource::GlobIterator::GlobIterator( recursive = blob_path_with_globs == "/**" ? true : false; filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - is_initialized = true; } RelativePathWithMetadata StorageAzureBlobSource::GlobIterator::next() @@ -1044,22 +996,8 @@ RelativePathWithMetadata StorageAzureBlobSource::GlobIterator::next() } index = 0; - if (!is_initialized) - { - filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, fs::path(container) / new_batch.front().relative_path, getContext()); - is_initialized = true; - } - if (filter_ast) - { - std::vector paths; - paths.reserve(new_batch.size()); - for (auto & path_with_metadata : new_batch) - paths.push_back(fs::path(container) / path_with_metadata.relative_path); - - VirtualColumnUtils::filterByPathOrFile(new_batch, paths, query, virtual_columns, getContext(), filter_ast); - } - else if (filter_dag) + if (filter_dag) { std::vector paths; paths.reserve(new_batch.size()); diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 30b91b7f85a..cc908fa3215 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -155,16 +155,6 @@ public: class GlobIterator : public IIterator { public: - GlobIterator( - AzureObjectStorage * object_storage_, - const std::string & container_, - String blob_path_with_globs_, - ASTPtr query_, - const NamesAndTypesList & virtual_columns_, - ContextPtr context_, - RelativePathsWithMetadata * outer_blobs_, - std::function file_progress_callback_ = {}); - GlobIterator( AzureObjectStorage * object_storage_, const std::string & container_, @@ -182,8 +172,6 @@ public: AzureObjectStorage * object_storage; std::string container; String blob_path_with_globs; - ASTPtr query; - ASTPtr filter_ast; ActionsDAGPtr filter_dag; NamesAndTypesList virtual_columns; @@ -198,7 +186,6 @@ public: void createFilterAST(const String & any_key); bool is_finished = false; - bool is_initialized = false; std::mutex next_mutex; std::function file_progress_callback; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 18acbfc7153..47d29c3c501 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -932,24 +932,6 @@ static std::chrono::seconds getLockTimeout(ContextPtr context) using StorageFilePtr = std::shared_ptr; - -StorageFileSource::FilesIterator::FilesIterator( - const Strings & files_, - std::optional archive_info_, - ASTPtr query, - const NamesAndTypesList & virtual_columns, - ContextPtr context_, - bool distributed_processing_) - : files(files_), archive_info(std::move(archive_info_)), distributed_processing(distributed_processing_), context(context_) -{ - ASTPtr filter_ast; - if (!distributed_processing && !archive_info && !files.empty() && !files[0].empty()) - filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, files[0], context_); - - if (filter_ast) - VirtualColumnUtils::filterByPathOrFile(files, files, query, virtual_columns, context_, filter_ast); -} - StorageFileSource::FilesIterator::FilesIterator( const Strings & files_, std::optional archive_info_, diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index ecb9e01b862..4f8cbfd4795 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -193,14 +193,6 @@ public: class FilesIterator { public: - explicit FilesIterator( - const Strings & files_, - std::optional archive_info_, - ASTPtr query, - const NamesAndTypesList & virtual_columns, - ContextPtr context_, - bool distributed_processing_ = false); - explicit FilesIterator( const Strings & files_, std::optional archive_info_, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 375a367bfab..88ea57e21cc 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -260,55 +260,6 @@ using OutputFormatPtr = std::shared_ptr; class StorageS3Source::DisclosedGlobIterator::Impl : WithContext { public: - Impl( - const S3::Client & client_, - const S3::URI & globbed_uri_, - ASTPtr & query_, - const NamesAndTypesList & virtual_columns_, - ContextPtr context_, - KeysWithInfo * read_keys_, - const S3Settings::RequestSettings & request_settings_, - std::function file_progress_callback_) - : WithContext(context_) - , client(client_.clone()) - , globbed_uri(globbed_uri_) - , query(query_) - , virtual_columns(virtual_columns_) - , read_keys(read_keys_) - , request_settings(request_settings_) - , list_objects_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, CurrentMetrics::StorageS3ThreadsScheduled, 1) - , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, "ListObjects")) - , file_progress_callback(file_progress_callback_) - { - if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) - throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Expression can not have wildcards inside bucket name"); - - const String key_prefix = globbed_uri.key.substr(0, globbed_uri.key.find_first_of("*?{")); - - /// We don't have to list bucket, because there is no asterisks. - if (key_prefix.size() == globbed_uri.key.size()) - { - buffer.emplace_back(std::make_shared(globbed_uri.key, std::nullopt)); - buffer_iter = buffer.begin(); - is_finished = true; - return; - } - - request.SetBucket(globbed_uri.bucket); - request.SetPrefix(key_prefix); - request.SetMaxKeys(static_cast(request_settings.list_object_keys_size)); - - outcome_future = listObjectsAsync(); - - matcher = std::make_unique(makeRegexpPatternFromGlobs(globbed_uri.key)); - if (!matcher->ok()) - throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, - "Cannot compile regex from glob ({}): {}", globbed_uri.key, matcher->error()); - - recursive = globbed_uri.key == "/**" ? true : false; - fillInternalBufferAssumeLocked(); - } - Impl( const S3::Client & client_, const S3::URI & globbed_uri_, @@ -357,7 +308,6 @@ public: fillInternalBufferAssumeLocked(); filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - is_initialized = true; } KeyWithInfoPtr next() @@ -475,22 +425,7 @@ private: return; } - if (!is_initialized) - { - filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, fs::path(globbed_uri.bucket) / temp_buffer.front()->key, getContext()); - is_initialized = true; - } - - if (filter_ast) - { - std::vector paths; - paths.reserve(temp_buffer.size()); - for (const auto & key_with_info : temp_buffer) - paths.push_back(fs::path(globbed_uri.bucket) / key_with_info->key); - - VirtualColumnUtils::filterByPathOrFile(temp_buffer, paths, query, virtual_columns, getContext(), filter_ast); - } - else if (filter_dag) + if (filter_dag) { std::vector paths; paths.reserve(temp_buffer.size()); @@ -539,8 +474,6 @@ private: S3::URI globbed_uri; ASTPtr query; NamesAndTypesList virtual_columns; - bool is_initialized{false}; - ASTPtr filter_ast; ActionsDAGPtr filter_dag; std::unique_ptr matcher; bool recursive{false}; @@ -556,19 +489,6 @@ private: std::function file_progress_callback; }; -StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( - const S3::Client & client_, - const S3::URI & globbed_uri_, - ASTPtr query, - const NamesAndTypesList & virtual_columns_, - ContextPtr context, - KeysWithInfo * read_keys_, - const S3Settings::RequestSettings & request_settings_, - std::function file_progress_callback_) - : pimpl(std::make_shared(client_, globbed_uri_, query, virtual_columns_, context, read_keys_, request_settings_, file_progress_callback_)) -{ -} - StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( const S3::Client & client_, const S3::URI & globbed_uri_, diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index dd7e0edb2d9..f63bf3a8e90 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -75,16 +75,6 @@ public: class DisclosedGlobIterator : public IIterator { public: - DisclosedGlobIterator( - const S3::Client & client_, - const S3::URI & globbed_uri_, - ASTPtr query, - const NamesAndTypesList & virtual_columns, - ContextPtr context, - KeysWithInfo * read_keys_ = nullptr, - const S3Settings::RequestSettings & request_settings_ = {}, - std::function progress_callback_ = {}); - DisclosedGlobIterator( const S3::Client & client_, const S3::URI & globbed_uri_, diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index ac17f880738..6ed535a0317 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -184,25 +184,6 @@ namespace class StorageURLSource::DisclosedGlobIterator::Impl { public: - Impl(const String & uri_, size_t max_addresses, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - { - uris = parseRemoteDescription(uri_, 0, uri_.size(), ',', max_addresses); - - ASTPtr filter_ast; - if (!uris.empty()) - filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, Poco::URI(uris[0]).getPath(), context); - - if (filter_ast) - { - std::vector paths; - paths.reserve(uris.size()); - for (const auto & uri : uris) - paths.push_back(Poco::URI(uri).getPath()); - - VirtualColumnUtils::filterByPathOrFile(uris, paths, query, virtual_columns, context, filter_ast); - } - } - Impl(const String & uri_, size_t max_addresses, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) { uris = parseRemoteDescription(uri_, 0, uri_.size(), ',', max_addresses); @@ -241,9 +222,6 @@ private: std::atomic_size_t index = 0; }; -StorageURLSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, size_t max_addresses, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - : pimpl(std::make_shared(uri, max_addresses, query, virtual_columns, context)) {} - StorageURLSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, size_t max_addresses, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) : pimpl(std::make_shared(uri, max_addresses, predicate, virtual_columns, context)) {} diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 1b2fb97cb28..c9e6f6311bf 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -139,7 +139,6 @@ public: class DisclosedGlobIterator { public: - DisclosedGlobIterator(const String & uri_, size_t max_addresses, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context); DisclosedGlobIterator(const String & uri_, size_t max_addresses, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context); String next(); From 1b20ce51624f996fc3995c5c511ecce2e6de872a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 2 Jan 2024 17:50:06 +0000 Subject: [PATCH 28/61] Cleanup --- src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/IStorageCluster.cpp | 16 ++++---- src/Storages/S3Queue/StorageS3Queue.cpp | 12 +++--- src/Storages/StorageAzureBlob.cpp | 7 ++-- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageFile.cpp | 50 +++++++------------------ src/Storages/StorageS3.cpp | 6 +-- src/Storages/StorageURL.cpp | 4 +- 8 files changed, 38 insertions(+), 61 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 430ecc7a585..c1c0f7d76bd 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -1,4 +1,3 @@ -#include "Processors/Sources/NullSource.h" #include "config.h" #if USE_HDFS @@ -16,6 +15,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp index c59b74255b2..6f42d8f855c 100644 --- a/src/Storages/IStorageCluster.cpp +++ b/src/Storages/IStorageCluster.cpp @@ -1,10 +1,7 @@ -#include "Storages/IStorageCluster.h" +#include -#include "Common/Exception.h" -#include "Core/QueryProcessingStage.h" -#include "Processors/QueryPlan/SourceStepWithFilter.h" -#include "Processors/Sources/NullSource.h" -#include "QueryPipeline/QueryPipelineBuilder.h" +#include +#include #include #include #include @@ -14,11 +11,14 @@ #include #include #include +#include +#include +#include +#include #include #include -#include #include -#include +#include #include #include #include diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 6d078e1aa1b..bc33e8cf2a9 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -1,7 +1,3 @@ -#include "Processors/QueryPlan/QueryPlan.h" -#include "Processors/QueryPlan/SourceStepWithFilter.h" -#include "Processors/Sources/NullSource.h" -#include "QueryPipeline/QueryPipelineBuilder.h" #include "config.h" #if USE_AWS_S3 @@ -10,11 +6,14 @@ #include #include #include +#include +#include #include #include #include -#include -#include +#include +#include +#include #include #include #include @@ -24,6 +23,7 @@ #include #include #include +#include #include diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 294a65c067f..4f63b144f43 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1,8 +1,4 @@ #include -#include "Processors/QueryPlan/QueryPlan.h" -#include "Processors/QueryPlan/SourceStepWithFilter.h" -#include "Processors/Sources/NullSource.h" - #if USE_AZURE_BLOB_STORAGE #include @@ -24,6 +20,9 @@ #include #include #include +#include +#include +#include #include #include diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c914388e55e..7ef2ff08827 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -30,7 +30,6 @@ #include #include #include -#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" #include #include @@ -92,6 +91,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 47d29c3c501..60e06291200 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -38,6 +38,8 @@ #include #include #include +#include +#include #include #include @@ -45,8 +47,6 @@ #include #include #include -#include -#include #include #include @@ -1330,25 +1330,15 @@ public: ReadFromFile( Block sample_block, std::shared_ptr storage_, - std::vector paths_, - std::optional archive_info_, - NamesAndTypesList virtual_columns_, - bool distributed_processing_, ReadFromFormatInfo info_, const bool need_only_count_, - size_t total_bytes_to_read_, ContextPtr context_, size_t max_block_size_, size_t num_streams_) : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) , storage(std::move(storage_)) - , paths(std::move(paths_)) - , archive_info(std::move(archive_info_)) - , virtual_columns(std::move(virtual_columns_)) - , distributed_processing(distributed_processing_) , info(std::move(info_)) , need_only_count(need_only_count_) - , total_bytes_to_read(total_bytes_to_read_) , context(std::move(context_)) , max_block_size(max_block_size_) , max_num_streams(num_streams_) @@ -1357,28 +1347,15 @@ public: private: std::shared_ptr storage; - - std::vector paths; - std::optional archive_info; - - NamesAndTypesList virtual_columns; - const bool distributed_processing; - ReadFromFormatInfo info; const bool need_only_count; - size_t total_bytes_to_read; - ContextPtr context; - size_t max_block_size; const size_t max_num_streams; std::shared_ptr files_iterator; - // FieldVectorPtr keys; - // bool all_scan = false; - void createIterator(const ActionsDAG::Node * predicate); }; @@ -1435,13 +1412,8 @@ void StorageFile::read( auto reading = std::make_unique( read_from_format_info.source_header, std::move(this_ptr), - paths, - archive_info, - virtual_columns, - distributed_processing, std::move(read_from_format_info), need_only_count, - total_bytes_to_read, context, max_block_size, num_streams); @@ -1454,7 +1426,13 @@ void ReadFromFile::createIterator(const ActionsDAG::Node * predicate) if (files_iterator) return; - files_iterator = std::make_shared(paths, archive_info, predicate, virtual_columns, context, distributed_processing); + files_iterator = std::make_shared( + storage->paths, + storage->archive_info, + predicate, + storage->virtual_columns, + context, + storage->distributed_processing); } void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) @@ -1464,10 +1442,10 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui size_t num_streams = max_num_streams; size_t files_to_read = 0; - if (archive_info) - files_to_read = archive_info->paths_to_archives.size(); + if (storage->archive_info) + files_to_read = storage->archive_info->paths_to_archives.size(); else - files_to_read = paths.size(); + files_to_read = storage->paths.size(); if (max_num_streams > files_to_read) num_streams = files_to_read; @@ -1478,8 +1456,8 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui /// Set total number of bytes to process. For progress bar. auto progress_callback = context->getFileProgressCallback(); - if (progress_callback && !archive_info) - progress_callback(FileProgress(0, total_bytes_to_read)); + if (progress_callback && !storage->archive_info) + progress_callback(FileProgress(0, storage->total_bytes_to_read)); for (size_t i = 0; i < num_streams; ++i) { diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 88ea57e21cc..3e1af2df4b0 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1,7 +1,4 @@ -#include "Processors/Sources/NullSource.h" #include "config.h" -#include -#include "Parsers/ASTCreateQuery.h" #if USE_AWS_S3 @@ -17,6 +14,7 @@ #include #include +#include #include #include @@ -43,6 +41,7 @@ #include #include #include +#include #include @@ -58,6 +57,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 6ed535a0317..3389ed1db86 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -26,6 +26,8 @@ #include #include #include +#include +#include #include #include @@ -34,8 +36,6 @@ #include #include #include -#include "Processors/QueryPlan/QueryPlan.h" -#include "Processors/QueryPlan/SourceStepWithFilter.h" #include #include From 4f99a8bc1f7f8a3d5e3ad9188ae649caefec8ed5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 2 Jan 2024 17:54:20 +0000 Subject: [PATCH 29/61] Remove more unused code. --- src/Storages/VirtualColumnUtils.cpp | 38 ----------------------------- src/Storages/VirtualColumnUtils.h | 19 --------------- 2 files changed, 57 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index b63b4e7cca7..e54528bbf01 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -424,44 +424,6 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const return block.getByName("_idx").column; } -ASTPtr createPathAndFileFilterAst(const ASTPtr & query, const NamesAndTypesList & virtual_columns, const String & path_example, const ContextPtr & context) -{ - if (!query || virtual_columns.empty()) - return {}; - - Block block; - for (const auto & column : virtual_columns) - { - if (column.name == "_file" || column.name == "_path") - block.insert({column.type->createColumn(), column.type, column.name}); - } - /// Create a block with one row to construct filter - /// Append "idx" column as the filter result - block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); - addPathAndFileToVirtualColumns(block, path_example, 0); - ASTPtr filter_ast; - prepareFilterBlockWithQuery(query, context, block, filter_ast); - return filter_ast; -} - -ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context, ASTPtr filter_ast) -{ - Block block; - for (const auto & column : virtual_columns) - { - if (column.name == "_file" || column.name == "_path") - block.insert({column.type->createColumn(), column.type, column.name}); - } - block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); - - for (size_t i = 0; i != paths.size(); ++i) - addPathAndFileToVirtualColumns(block, paths[i], i); - - filterBlockWithQuery(query, block, context, filter_ast); - - return block.getByName("_idx").column; -} - void addRequestedPathFileAndSizeVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename) { diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 6e1af0995cc..3c07e33a177 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -58,25 +58,6 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) NamesAndTypesList getPathFileAndSizeVirtualsForStorage(NamesAndTypesList storage_columns); -ASTPtr createPathAndFileFilterAst(const ASTPtr & query, const NamesAndTypesList & virtual_columns, const String & path_example, const ContextPtr & context); - -ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context, ASTPtr filter_ast); - -template -void filterByPathOrFile(std::vector & sources, const std::vector & paths, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context, ASTPtr filter_ast) -{ - auto indexes_column = getFilterByPathAndFileIndexes(paths, query, virtual_columns, context, filter_ast); - const auto & indexes = typeid_cast(*indexes_column).getData(); - if (indexes.size() == sources.size()) - return; - - std::vector filtered_sources; - filtered_sources.reserve(indexes.size()); - for (auto index : indexes) - filtered_sources.emplace_back(std::move(sources[index])); - sources = std::move(filtered_sources); -} - ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context); From 9c25cb6692cfdcf410c9d735a77e9c2eb01fff78 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 2 Jan 2024 18:08:04 +0000 Subject: [PATCH 30/61] Cleanup --- src/Storages/HDFS/StorageHDFS.cpp | 25 ++++--------------------- src/Storages/HDFS/StorageHDFS.h | 3 +-- src/Storages/StorageURL.cpp | 16 ---------------- 3 files changed, 5 insertions(+), 39 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index c1c0f7d76bd..974b2bb68cf 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -411,7 +411,6 @@ ColumnsDescription StorageHDFS::getTableStructureFromData( class HDFSSource::DisclosedGlobIterator::Impl { public: - Impl(const String & uri, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) { const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); @@ -854,10 +853,6 @@ public: ReadFromHDFS( Block sample_block, - std::vector uris_, - bool distributed_processing_, - NamesAndTypesList virtual_columns_, - bool is_path_with_globs_, ReadFromFormatInfo info_, bool need_only_count_, std::shared_ptr storage_, @@ -865,10 +860,6 @@ public: size_t max_block_size_, size_t num_streams_) : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) - , uris(std::move(uris_)) - , distributed_processing(distributed_processing_) - , virtual_columns(std::move(virtual_columns_)) - , is_path_with_globs(is_path_with_globs_) , info(std::move(info_)) , need_only_count(need_only_count_) , storage(std::move(storage_)) @@ -879,10 +870,6 @@ public: } private: - std::vector uris; - const bool distributed_processing; - NamesAndTypesList virtual_columns; - bool is_path_with_globs; ReadFromFormatInfo info; const bool need_only_count; std::shared_ptr storage; @@ -924,10 +911,6 @@ void StorageHDFS::read( auto reading = std::make_unique( read_from_format_info.source_header, - uris, - distributed_processing, - virtual_columns, - is_path_with_globs, std::move(read_from_format_info), need_only_count, std::move(this_ptr), @@ -943,17 +926,17 @@ void ReadFromHDFS::createIterator(const ActionsDAG::Node * predicate) if (iterator_wrapper) return; - if (distributed_processing) + if (storage->distributed_processing) { iterator_wrapper = std::make_shared( [callback = context->getReadTaskCallback()]() -> StorageHDFS::PathWithInfo { return StorageHDFS::PathWithInfo{callback(), std::nullopt}; }); } - else if (is_path_with_globs) + else if (storage->is_path_with_globs) { /// Iterate through disclosed globs and make a source for each file - auto glob_iterator = std::make_shared(uris[0], predicate, virtual_columns, context); + auto glob_iterator = std::make_shared(storage->uris[0], predicate, storage->virtual_columns, context); iterator_wrapper = std::make_shared([glob_iterator]() { return glob_iterator->next(); @@ -961,7 +944,7 @@ void ReadFromHDFS::createIterator(const ActionsDAG::Node * predicate) } else { - auto uris_iterator = std::make_shared(uris, predicate, virtual_columns, context); + auto uris_iterator = std::make_shared(storage->uris, predicate, storage->virtual_columns, context); iterator_wrapper = std::make_shared([uris_iterator]() { return uris_iterator->next(); diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 9d9a857bf4e..f1f0019d3e0 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -94,6 +94,7 @@ public: protected: friend class HDFSSource; + friend class ReadFromHDFS; private: std::vector uris; @@ -144,7 +145,6 @@ public: UInt64 max_block_size_, std::shared_ptr file_iterator_, bool need_only_count_); - //const SelectQueryInfo & query_info_); String getName() const override; @@ -163,7 +163,6 @@ private: ColumnsDescription columns_description; bool need_only_count; size_t total_rows_in_file = 0; - //SelectQueryInfo query_info; std::unique_ptr read_buf; std::shared_ptr input_format; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 3389ed1db86..36219d13a45 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -896,15 +896,10 @@ public: : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) , storage(std::move(storage_)) , uri_options(uri_options_) - // , paths(std::move(paths_)) - // , archive_info(std::move(archive_info_)) - // , virtual_columns(std::move(virtual_columns_)) - // , distributed_processing(distributed_processing_) , info(std::move(info_)) , need_only_count(need_only_count_) , read_uri_params(std::move(read_uri_params_)) , read_post_data_callback(std::move(read_post_data_callback_)) - // , total_bytes_to_read(total_bytes_to_read_) , context(std::move(context_)) , max_block_size(max_block_size_) , num_streams(num_streams_) @@ -915,19 +910,11 @@ private: std::shared_ptr storage; std::vector * uri_options; - // std::vector paths; - // std::optional archive_info; - - // NamesAndTypesList virtual_columns; - // const bool distributed_processing; - ReadFromFormatInfo info; const bool need_only_count; std::vector> read_uri_params; std::function read_post_data_callback; - // size_t total_bytes_to_read; - ContextPtr context; size_t max_block_size; @@ -937,9 +924,6 @@ private: bool is_url_with_globs = false; bool is_empty_glob = false; - // FieldVectorPtr keys; - // bool all_scan = false; - void createIterator(const ActionsDAG::Node * predicate); }; From 30876b159afb21fc6a251c65b92d9ae793b59594 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 2 Jan 2024 20:56:19 +0100 Subject: [PATCH 31/61] fix --- .../0_stateless/01175_distributed_ddl_output_mode_long.sh | 2 +- .../0_stateless/02447_drop_database_replica.reference | 4 ---- tests/queries/0_stateless/02447_drop_database_replica.sh | 5 ++--- 3 files changed, 3 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh index d2695e602c5..12e142adda9 100755 --- a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh +++ b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh @@ -33,7 +33,7 @@ function run_until_out_contains() done } -RAND_COMMENT="01175_DDL_$RANDOM" +RAND_COMMENT="01175_DDL_$CLICKHOUSE_DATABASE" LOG_COMMENT="${CLICKHOUSE_LOG_COMMENT}_$RAND_COMMENT" CLICKHOUSE_CLIENT_WITH_SETTINGS=${CLICKHOUSE_CLIENT/--log_comment ${CLICKHOUSE_LOG_COMMENT}/--log_comment ${LOG_COMMENT}} diff --git a/tests/queries/0_stateless/02447_drop_database_replica.reference b/tests/queries/0_stateless/02447_drop_database_replica.reference index 7be5dde1998..1af3ee244f1 100644 --- a/tests/queries/0_stateless/02447_drop_database_replica.reference +++ b/tests/queries/0_stateless/02447_drop_database_replica.reference @@ -12,9 +12,6 @@ t 2 rdb_default 1 1 s1 r1 1 2 -s1 r1 OK 2 0 -s1 r2 QUEUED 2 0 -s2 r1 QUEUED 2 0 2 s1 r1 OK 2 0 s1 r2 QUEUED 2 0 @@ -27,5 +24,4 @@ rdb_default 1 2 s1 r2 0 t t2 t3 -t4 rdb_default_4 1 1 s1 r1 1 diff --git a/tests/queries/0_stateless/02447_drop_database_replica.sh b/tests/queries/0_stateless/02447_drop_database_replica.sh index d12f173f388..fb89db5045b 100755 --- a/tests/queries/0_stateless/02447_drop_database_replica.sh +++ b/tests/queries/0_stateless/02447_drop_database_replica.sh @@ -33,9 +33,8 @@ $CLICKHOUSE_CLIENT -q "select cluster, shard_num, replica_num, database_shard_na $CLICKHOUSE_CLIENT -q "system drop database replica 's1|r1' from database $db2" 2>&1| grep -Fac "is active, cannot drop it" # Also check that it doesn't exceed distributed_ddl_task_timeout waiting for inactive replicas -timeout 10s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=throw_only_active -q "create table $db.t2 (n int) engine=Log" 2>/dev/null | sort -timeout 10s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=throw_only_active -q "create table $db.t3 (n int) engine=Log" 2>&1| grep -Fac "TIMEOUT_EXCEEDED" -timeout 10s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=null_status_on_timeout_only_active -q "create table $db.t4 (n int) engine=Log" | sort +timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=throw_only_active -q "create table $db.t2 (n int) engine=Log" 2>&1| grep -Fac "TIMEOUT_EXCEEDED" +timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=null_status_on_timeout_only_active -q "create table $db.t3 (n int) engine=Log" | sort $CLICKHOUSE_CLIENT -q "detach database $db3" $CLICKHOUSE_CLIENT -q "system drop database replica 'r1' from shard 's2' from database $db" From eeed23b1bc2b789c0d3097595540b16ef7e788b0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 3 Jan 2024 09:45:25 +0000 Subject: [PATCH 32/61] Fix sanitizer assert. --- src/Storages/StorageURL.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 36219d13a45..9ace7775d4b 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -1035,12 +1035,12 @@ void ReadFromURL::createIterator(const ActionsDAG::Node * predicate) } else { - iterator_wrapper = std::make_shared([&, max_addresses, done = false]() mutable + iterator_wrapper = std::make_shared([max_addresses, done = false, &uri = storage->uri]() mutable { if (done) return StorageURLSource::FailoverOptions{}; done = true; - return getFailoverOptions(storage->uri, max_addresses); + return getFailoverOptions(uri, max_addresses); }); num_streams = 1; } From 91fc3b3456590ffc9577b080357513e651303b1d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 3 Jan 2024 10:54:42 +0000 Subject: [PATCH 33/61] Linter --- src/Common/findExtreme.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/findExtreme.cpp b/src/Common/findExtreme.cpp index e1f1e199d56..032ac75b79b 100644 --- a/src/Common/findExtreme.cpp +++ b/src/Common/findExtreme.cpp @@ -20,7 +20,7 @@ struct MaxComparator MULTITARGET_FUNCTION_AVX2_SSE42( MULTITARGET_FUNCTION_HEADER(template static std::optional NO_INLINE), findExtremeImpl, - MULTITARGET_FUNCTION_BODY((const T * __restrict ptr, const UInt8 * __restrict condition_map [[maybe_unused]], size_t row_begin, size_t row_end) + MULTITARGET_FUNCTION_BODY((const T * __restrict ptr, const UInt8 * __restrict condition_map [[maybe_unused]], size_t row_begin, size_t row_end) /// NOLINT { size_t count = row_end - row_begin; ptr += row_begin; From 89beb32e646e54012171d4388d874ba8b80fc839 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 3 Jan 2024 13:10:14 +0100 Subject: [PATCH 34/61] Edit docs for toWeek() --- docs/en/sql-reference/functions/date-time-functions.md | 4 +++- docs/ru/sql-reference/functions/date-time-functions.md | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 0261589b968..5622097537e 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1483,7 +1483,9 @@ For mode values with a meaning of “with 4 or more days this year,” weeks are - Otherwise, it is the last week of the previous year, and the next week is week 1. -For mode values with a meaning of “contains January 1”, the week contains January 1 is week 1. It does not matter how many days in the new year the week contained, even if it contained only one day. +For mode values with a meaning of “contains January 1”, the week contains January 1 is week 1. +It does not matter how many days in the new year the week contained, even if it contained only one day. +I.e. if the last week of December contains January 1 of the next year, it will be week 1 of the next year. **Syntax** diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index fa5728a097d..cbbb456aa80 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -578,7 +578,9 @@ SELECT - В противном случае это последняя неделя предыдущего года, а следующая неделя - неделя 1. -Для режимов со значением «содержит 1 января», неделя 1 – это неделя содержащая 1 января. Не имеет значения, сколько дней в новом году содержала неделя, даже если она содержала только один день. +Для режимов со значением «содержит 1 января», неделя 1 – это неделя, содержащая 1 января. +Не имеет значения, сколько дней нового года содержит эта неделя, даже если она содержит только один день. +Так, если последняя неделя декабря содержит 1 января следующего года, то она считается неделей 1 следующего года. **Пример** From be825b129053f1c47762e08142ffebf5761c1df8 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 3 Jan 2024 12:20:08 +0000 Subject: [PATCH 35/61] fix segfault when graphite table does not have agg function Signed-off-by: Duc Canh Le --- src/Processors/Merges/Algorithms/Graphite.h | 7 +- .../config/config.d/graphite_alternative.xml | 24 +++++++ ...cated_merge_parameters_must_consistent.sql | 70 ++++++++++++++++--- 3 files changed, 92 insertions(+), 9 deletions(-) diff --git a/src/Processors/Merges/Algorithms/Graphite.h b/src/Processors/Merges/Algorithms/Graphite.h index 692e36d2eae..04bb4548c14 100644 --- a/src/Processors/Merges/Algorithms/Graphite.h +++ b/src/Processors/Merges/Algorithms/Graphite.h @@ -127,7 +127,12 @@ struct Pattern { hash.update(rule_type); hash.update(regexp_str); - hash.update(function->getName()); + if (function) + { + hash.update(function->getName()); + for (const auto & p : function->getParameters()) + hash.update(toString(p)); + } for (const auto & r : retentions) { hash.update(r.age); diff --git a/tests/config/config.d/graphite_alternative.xml b/tests/config/config.d/graphite_alternative.xml index 1a00de52af5..6c0bd13ce43 100644 --- a/tests/config/config.d/graphite_alternative.xml +++ b/tests/config/config.d/graphite_alternative.xml @@ -26,4 +26,28 @@ + + Version + + sum + + 0 + 600 + + + 17280 + 6000 + + + + + 0 + 600 + + + 17280 + 6000 + + + diff --git a/tests/queries/0_stateless/02910_replicated_merge_parameters_must_consistent.sql b/tests/queries/0_stateless/02910_replicated_merge_parameters_must_consistent.sql index 3c1bec4fb3f..0f452105e6d 100644 --- a/tests/queries/0_stateless/02910_replicated_merge_parameters_must_consistent.sql +++ b/tests/queries/0_stateless/02910_replicated_merge_parameters_must_consistent.sql @@ -8,13 +8,22 @@ CREATE TABLE t ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t/', 'r1', legacy_ver) ORDER BY id; -CREATE TABLE t_r +CREATE TABLE t_r_ok +( + `id` UInt64, + `val` String, + `legacy_ver` UInt64, +) +ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t/', 'r2', legacy_ver) +ORDER BY id; + +CREATE TABLE t_r_error ( `id` UInt64, `val` String, `legacy_ver` UInt64 ) -ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t/', 'r2') +ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t/', 'r3') ORDER BY id; -- { serverError METADATA_MISMATCH } CREATE TABLE t2 @@ -27,14 +36,24 @@ CREATE TABLE t2 ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t2/', 'r1', legacy_ver) ORDER BY id; -CREATE TABLE t2_r +CREATE TABLE t2_r_ok ( `id` UInt64, `val` String, `legacy_ver` UInt64, `deleted` UInt8 ) -ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t2/', 'r2', legacy_ver, deleted) +ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t2/', 'r2', legacy_ver) +ORDER BY id; + +CREATE TABLE t2_r_error +( + `id` UInt64, + `val` String, + `legacy_ver` UInt64, + `deleted` UInt8 +) +ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t2/', 'r3', legacy_ver, deleted) ORDER BY id; -- { serverError METADATA_MISMATCH } CREATE TABLE t3 @@ -46,13 +65,23 @@ CREATE TABLE t3 ENGINE = ReplicatedSummingMergeTree('/tables/{database}/t3/', 'r1', metrics1) ORDER BY key; -CREATE TABLE t3_r +CREATE TABLE t3_r_ok ( `key` UInt64, `metrics1` UInt64, `metrics2` UInt64 ) -ENGINE = ReplicatedSummingMergeTree('/tables/{database}/t3/', 'r2', metrics2) +ENGINE = ReplicatedSummingMergeTree('/tables/{database}/t3/', 'r2', metrics1) +ORDER BY key; + + +CREATE TABLE t3_r_error +( + `key` UInt64, + `metrics1` UInt64, + `metrics2` UInt64 +) +ENGINE = ReplicatedSummingMergeTree('/tables/{database}/t3/', 'r3', metrics2) ORDER BY key; -- { serverError METADATA_MISMATCH } CREATE TABLE t4 @@ -67,7 +96,7 @@ CREATE TABLE t4 ENGINE = ReplicatedGraphiteMergeTree('/tables/{database}/t4/', 'r1', 'graphite_rollup') ORDER BY key; -CREATE TABLE t4_r +CREATE TABLE t4_r_ok ( `key` UInt32, `Path` String, @@ -76,5 +105,30 @@ CREATE TABLE t4_r `Version` UInt32, `col` UInt64 ) -ENGINE = ReplicatedGraphiteMergeTree('/tables/{database}/t4/', 'r2', 'graphite_rollup_alternative') +ENGINE = ReplicatedGraphiteMergeTree('/tables/{database}/t4/', 'r2', 'graphite_rollup') +ORDER BY key; + +CREATE TABLE t4_r_error +( + `key` UInt32, + `Path` String, + `Time` DateTime('UTC'), + `Value` Float64, + `Version` UInt32, + `col` UInt64 +) +ENGINE = ReplicatedGraphiteMergeTree('/tables/{database}/t4/', 'r3', 'graphite_rollup_alternative') ORDER BY key; -- { serverError METADATA_MISMATCH } + +-- https://github.com/ClickHouse/ClickHouse/issues/58451 +CREATE TABLE t4_r_error_2 +( + `key` UInt32, + `Path` String, + `Time` DateTime('UTC'), + `Value` Float64, + `Version` UInt32, + `col` UInt64 +) +ENGINE = ReplicatedGraphiteMergeTree('/tables/{database}/t4/', 'r4', 'graphite_rollup_alternative_no_function') +ORDER BY key; -- { serverError METADATA_MISMATCH } \ No newline at end of file From 1f960a32de2f63012fcba3f4cb1b28ebf596d64f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 3 Jan 2024 14:04:30 +0100 Subject: [PATCH 36/61] Fix OSX build --- src/Common/iota.cpp | 3 +++ src/Common/iota.h | 16 ++++++++++++++-- src/Common/tests/gtest_hash_table.cpp | 2 +- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 2 +- 4 files changed, 19 insertions(+), 4 deletions(-) diff --git a/src/Common/iota.cpp b/src/Common/iota.cpp index 385d3b22207..98f18eb195b 100644 --- a/src/Common/iota.cpp +++ b/src/Common/iota.cpp @@ -30,4 +30,7 @@ void iota(T * begin, size_t count, T first_value) template void iota(UInt8 * begin, size_t count, UInt8 first_value); template void iota(UInt32 * begin, size_t count, UInt32 first_value); template void iota(UInt64 * begin, size_t count, UInt64 first_value); +#if defined(OS_DARWIN) +template void iota(size_t * begin, size_t count, size_t first_value); +#endif } diff --git a/src/Common/iota.h b/src/Common/iota.h index 485df4bd4f0..7910274d15d 100644 --- a/src/Common/iota.h +++ b/src/Common/iota.h @@ -10,13 +10,25 @@ namespace DB { /// Make sure to add any new type to the extern declaration at the end of the file and instantiate it in iota.cpp + template -concept iota_supported_types = (is_any_of); +concept iota_supported_types = (is_any_of< + T, + UInt8, + UInt32, + UInt64 +#if defined(OS_DARWIN) + , + size_t +#endif + >); template void iota(T * begin, size_t count, T first_value); extern template void iota(UInt8 * begin, size_t count, UInt8 first_value); extern template void iota(UInt32 * begin, size_t count, UInt32 first_value); extern template void iota(UInt64 * begin, size_t count, UInt64 first_value); - +#if defined(OS_DARWIN) +extern template void iota(size_t * begin, size_t count, size_t first_value); +#endif } diff --git a/src/Common/tests/gtest_hash_table.cpp b/src/Common/tests/gtest_hash_table.cpp index ab7c3872170..ae432de7766 100644 --- a/src/Common/tests/gtest_hash_table.cpp +++ b/src/Common/tests/gtest_hash_table.cpp @@ -21,7 +21,7 @@ namespace std::vector getVectorWithNumbersUpToN(size_t n) { std::vector res(n); - iota(res.data(), res.size(), size_t(0)); + iota(res.data(), res.size(), UInt64(0)); return res; } diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 329497d66d3..5ccde0ba5bc 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -41,7 +41,7 @@ protected: auto column = ColumnUInt64::create(block_size); ColumnUInt64::Container & vec = column->getData(); - size_t curr = next; /// The local variable for some reason works faster (>20%) than member of class. + UInt64 curr = next; /// The local variable for some reason works faster (>20%) than member of class. UInt64 * pos = vec.data(); /// This also accelerates the code. UInt64 * end = &vec[block_size]; iota(pos, static_cast(end - pos), curr); From c8acc7c2d1d51c39c3a20dbcebc2eb03d49f0994 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 3 Jan 2024 14:44:00 +0100 Subject: [PATCH 37/61] Fix build --- src/Storages/StorageMaterializedView.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 59f1d5eee1b..934d57d40e2 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -72,7 +72,7 @@ public: StoragePtr getTargetTable() const; StoragePtr tryGetTargetTable() const; - StorageID getTargetTableId() const { return target_table_id; } + StorageID getTargetTableId() const; /// Get the virtual column of the target table; NamesAndTypesList getVirtuals() const override; From 31254826314fffb56f02a486297f8ba54a55173d Mon Sep 17 00:00:00 2001 From: Mark Needham Date: Wed, 3 Jan 2024 14:25:03 +0000 Subject: [PATCH 38/61] Add output_format_decimal_trailing_zeros setting for trailing spaces --- .../functions/rounding-functions.md | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index 84839c2489c..3ede66cf316 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -53,7 +53,7 @@ The rounded number of the same type as the input number. **Example of use with Float** ``` sql -SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3 +SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3; ``` ``` text @@ -67,7 +67,22 @@ SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3 **Example of use with Decimal** ``` sql -SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3 +SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3; +``` + +``` text +┌───x─┬─round(CAST(divide(number, 2), 'Decimal(10, 4)'))─┐ +│ 0 │ 0 │ +│ 0.5 │ 1 │ +│ 1 │ 1 │ +└─────┴──────────────────────────────────────────────────┘ +``` + +If you want to keep the trailing zeros, you need to enable `output_format_decimal_trailing_zeros` + +``` sql +SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3 settings output_format_decimal_trailing_zeros=1; + ``` ``` text From 7ee1697971e310d29aa00b4627f415b74b47b748 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 3 Jan 2024 17:16:45 +0000 Subject: [PATCH 39/61] Reduce setup time of min_max_index.xml --- tests/performance/min_max_index.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/min_max_index.xml b/tests/performance/min_max_index.xml index b7b5d4fb991..518696144e2 100644 --- a/tests/performance/min_max_index.xml +++ b/tests/performance/min_max_index.xml @@ -1,7 +1,7 @@ CREATE TABLE index_test (z UInt32, INDEX i_x (mortonDecode(2, z).1) TYPE minmax, INDEX i_y (mortonDecode(2, z).2) TYPE minmax) ENGINE = MergeTree ORDER BY z - INSERT INTO index_test SELECT number FROM numbers(0x100000000) WHERE rand() % 3 = 1 + INSERT INTO index_test SELECT number * 10 FROM numbers_mt(toUInt64(0x100000000 / 10)) SETTINGS max_insert_threads=8 = 20000 AND mortonDecode(2, z).1 <= 20100 AND mortonDecode(2, z).2 >= 10000 AND mortonDecode(2, z).2 <= 10100 From b8305e1a6e976cb040454089e57e6db97310d0e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 3 Jan 2024 17:19:44 +0000 Subject: [PATCH 40/61] Make test more reasonable --- tests/performance/group_by_sundy_li.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/performance/group_by_sundy_li.xml b/tests/performance/group_by_sundy_li.xml index 694fafcbbcd..46f659d9cc0 100644 --- a/tests/performance/group_by_sundy_li.xml +++ b/tests/performance/group_by_sundy_li.xml @@ -16,10 +16,10 @@ ORDER BY (d, n) - insert into a select '2000-01-01', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(100000000) - insert into a select '2000-01-02', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(100000000) - insert into a select '2000-01-03', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(100000000) - insert into a select '2000-01-04', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(100000000) + insert into a select '2000-01-01', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(10000000) + insert into a select '2000-01-02', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(10000000) + insert into a select '2000-01-03', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(10000000) + insert into a select '2000-01-04', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(10000000) OPTIMIZE TABLE a FINAL From 910b3385841297e442f6d349244db0052cc1c3e1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 3 Jan 2024 17:24:15 +0000 Subject: [PATCH 41/61] Reduce polymorphic_parts_m --- tests/performance/polymorphic_parts_l.xml | 4 ++-- tests/performance/polymorphic_parts_m.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/performance/polymorphic_parts_l.xml b/tests/performance/polymorphic_parts_l.xml index d2ae9417bf7..66c5b73caa8 100644 --- a/tests/performance/polymorphic_parts_l.xml +++ b/tests/performance/polymorphic_parts_l.xml @@ -25,8 +25,8 @@ - INSERT INTO hits_wide(UserID) SELECT rand() FROM numbers(100000) - INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(100000) + INSERT INTO hits_wide(UserID) SELECT rand() FROM numbers(100000) + INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(100000) INSERT INTO hits_buffer(UserID) SELECT rand() FROM numbers(100000) DROP TABLE IF EXISTS hits_wide diff --git a/tests/performance/polymorphic_parts_m.xml b/tests/performance/polymorphic_parts_m.xml index 54a81def55e..0a44038ffbd 100644 --- a/tests/performance/polymorphic_parts_m.xml +++ b/tests/performance/polymorphic_parts_m.xml @@ -25,8 +25,8 @@ - INSERT INTO hits_wide(UserID) SELECT rand() FROM numbers(10000) - INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(100000) + INSERT INTO hits_wide(UserID) SELECT rand() FROM numbers(10000) + INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(10000) INSERT INTO hits_buffer(UserID) SELECT rand() FROM numbers(10000) DROP TABLE IF EXISTS hits_wide From c223ae56d33723e52d331a19eb05d70b209792a8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 3 Jan 2024 17:29:30 +0000 Subject: [PATCH 42/61] Reduce the size of decimal_parse --- tests/performance/decimal_parse.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/decimal_parse.xml b/tests/performance/decimal_parse.xml index 19e940b13df..966363d6fec 100644 --- a/tests/performance/decimal_parse.xml +++ b/tests/performance/decimal_parse.xml @@ -1,3 +1,3 @@ - SELECT count() FROM zeros(10000000) WHERE NOT ignore(toDecimal32OrZero(toString(rand() % 10000), 5)) + SELECT count() FROM zeros(3000000) WHERE NOT ignore(toDecimal32OrZero(toString(rand() % 10000), 5)) From c1953206123ba0d8337212596ca64cf220365bc3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 3 Jan 2024 17:31:55 +0000 Subject: [PATCH 43/61] Reduce the size of join_used_flags.xml --- tests/performance/join_used_flags.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/join_used_flags.xml b/tests/performance/join_used_flags.xml index cd2073ee106..70b0b45391d 100644 --- a/tests/performance/join_used_flags.xml +++ b/tests/performance/join_used_flags.xml @@ -1,6 +1,6 @@ CREATE TABLE test_join_used_flags (i64 Int64, i32 Int32) ENGINE = Memory - INSERT INTO test_join_used_flags SELECT number AS i64, rand32() AS i32 FROM numbers(20000000) + INSERT INTO test_join_used_flags SELECT number AS i64, rand32() AS i32 FROM numbers_mt(3000000) SELECT l.i64, r.i64, l.i32, r.i32 FROM test_join_used_flags l RIGHT JOIN test_join_used_flags r USING i64 format Null DROP TABLE IF EXISTS test_join_used_flags From d06de83ac14dd8aab015868c84ac341799be7294 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 3 Jan 2024 17:44:28 +0000 Subject: [PATCH 44/61] Fix KeyCondition for file/url/s3 --- src/Processors/SourceWithKeyCondition.h | 34 ++++++++----------- src/Storages/StorageFile.cpp | 7 ++-- src/Storages/StorageS3.cpp | 7 ++-- src/Storages/StorageURL.cpp | 7 ++-- src/Storages/VirtualColumnUtils.cpp | 34 +++++++++++++++++++ .../02725_parquet_preserve_order.reference | 4 +-- 6 files changed, 65 insertions(+), 28 deletions(-) diff --git a/src/Processors/SourceWithKeyCondition.h b/src/Processors/SourceWithKeyCondition.h index 9e641cc8c51..c9617d3e73e 100644 --- a/src/Processors/SourceWithKeyCondition.h +++ b/src/Processors/SourceWithKeyCondition.h @@ -18,31 +18,25 @@ protected: void setKeyConditionImpl(const SelectQueryInfo & query_info, ContextPtr context, const Block & keys) { - if (!context->getSettingsRef().allow_experimental_analyzer) - { - key_condition = std::make_shared( - query_info, - context, - keys.getNames(), - std::make_shared(std::make_shared(keys.getColumnsWithTypeAndName()))); - } + key_condition = std::make_shared( + query_info, + context, + keys.getNames(), + std::make_shared(std::make_shared(keys.getColumnsWithTypeAndName()))); } void setKeyConditionImpl(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context, const Block & keys) { - if (context->getSettingsRef().allow_experimental_analyzer) - { - std::unordered_map node_name_to_input_column; - for (const auto & column : keys.getColumnsWithTypeAndName()) - node_name_to_input_column.insert({column.name, column}); + std::unordered_map node_name_to_input_column; + for (const auto & column : keys.getColumnsWithTypeAndName()) + node_name_to_input_column.insert({column.name, column}); - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_column, context); - key_condition = std::make_shared( - filter_actions_dag, - context, - keys.getNames(), - std::make_shared(std::make_shared(keys.getColumnsWithTypeAndName()))); - } + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_column, context); + key_condition = std::make_shared( + filter_actions_dag, + context, + keys.getNames(), + std::make_shared(std::make_shared(keys.getColumnsWithTypeAndName()))); } public: diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 60e06291200..f3917b878d6 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1469,14 +1469,17 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui if (storage->has_peekable_read_buffer_from_fd.exchange(false)) read_buffer = std::move(storage->peekable_read_buffer_from_fd); - pipes.emplace_back(std::make_shared( + auto source = std::make_shared( info, storage, context, max_block_size, files_iterator, std::move(read_buffer), - need_only_count)); + need_only_count); + + source->setKeyCondition(filter_nodes.nodes, context); + pipes.emplace_back(std::move(source)); } auto pipe = Pipe::unitePipes(std::move(pipes)); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index b6d96e21e33..ce49be32120 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1255,7 +1255,7 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, pipes.reserve(num_streams); for (size_t i = 0; i < num_streams; ++i) { - pipes.emplace_back(std::make_shared( + auto source = std::make_shared( read_from_format_info, query_configuration.format, storage.getName(), @@ -1270,7 +1270,10 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, query_configuration.url.uri.getHost() + std::to_string(query_configuration.url.uri.getPort()), iterator_wrapper, max_parsing_threads, - need_only_count)); + need_only_count); + + source->setKeyCondition(filter_nodes.nodes, local_context); + pipes.emplace_back(std::move(source)); } auto pipe = Pipe::unitePipes(std::move(pipes)); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 9ace7775d4b..c0e4be36202 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -1064,7 +1064,7 @@ void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const Buil for (size_t i = 0; i < num_streams; ++i) { - pipes.emplace_back(std::make_shared( + auto source = std::make_shared( info, iterator_wrapper, storage->getReadMethod(), @@ -1080,7 +1080,10 @@ void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const Buil storage->headers, read_uri_params, is_url_with_globs, - need_only_count)); + need_only_count); + + source->setKeyCondition(filter_nodes.nodes, context); + pipes.emplace_back(std::move(source)); } if (uri_options) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 20e9a5ea174..76138bbea87 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -36,7 +36,10 @@ #include #include #include +#include "Functions/FunctionsLogical.h" #include "Functions/IFunction.h" +#include "Functions/IFunctionAdaptors.h" +#include "Functions/indexHint.h" #include #include #include @@ -519,6 +522,37 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( return &node_copy; } + else if (node->function_base->getName() == "indexHint") + { + if (const auto * adaptor = typeid_cast(node->function_base.get())) + { + if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) + { + auto index_hint_dag = index_hint->getActions()->clone(); + ActionsDAG::NodeRawConstPtrs atoms; + for (const auto & output : index_hint_dag->getOutputs()) + if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes)) + atoms.push_back(child_copy); + + if (!atoms.empty()) + { + const auto * res = atoms.at(0); + + if (atoms.size() > 1) + { + FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); + res = &index_hint_dag->addFunction(func_builder_and, atoms, {}); + } + + if (!res->result_type->equals(*node->result_type)) + res = &index_hint_dag->addCast(*res, node->result_type, {}); + + additional_nodes.splice(additional_nodes.end(), ActionsDAG::detachNodes(std::move(*index_hint_dag))); + return res; + } + } + } + } } if (!canEvaluateSubtree(node, allowed_inputs)) diff --git a/tests/queries/0_stateless/02725_parquet_preserve_order.reference b/tests/queries/0_stateless/02725_parquet_preserve_order.reference index e9c8f99bb33..3f410c13ec4 100644 --- a/tests/queries/0_stateless/02725_parquet_preserve_order.reference +++ b/tests/queries/0_stateless/02725_parquet_preserve_order.reference @@ -3,10 +3,10 @@ 2 (Expression) ExpressionTransform - (ReadFromStorage) + (ReadFromFile) File 0 → 1 (Expression) ExpressionTransform × 2 - (ReadFromStorage) + (ReadFromFile) Resize 1 → 2 File 0 → 1 From 66d2db52832a81aea43cda66a500d8b3369547ef Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 4 Jan 2024 00:27:04 +0100 Subject: [PATCH 45/61] New parallel replicas coordinator implementation (#57968) --- src/Common/ProfileEvents.cpp | 12 + src/Core/Settings.h | 1 + .../ClusterProxy/executeQuery.cpp | 3 +- .../MergeTreeReadPoolParallelReplicas.cpp | 11 +- .../MergeTreeReadPoolParallelReplicas.h | 1 + .../ParallelReplicasReadingCoordinator.cpp | 810 +++++++++++++----- .../ParallelReplicasReadingCoordinator.h | 4 +- .../__init__.py | 0 .../configs/remote_servers.xml | 32 + .../test.py | 156 ++++ .../configs/remote_servers.xml | 22 - .../test.py | 156 ---- .../__init__.py | 0 .../configs/remote_servers.xml | 22 - .../test.py | 140 --- 15 files changed, 817 insertions(+), 553 deletions(-) rename tests/integration/{test_parallel_replicas_distributed_read_from_all => test_parallel_replicas_all_marks_read}/__init__.py (100%) create mode 100644 tests/integration/test_parallel_replicas_all_marks_read/configs/remote_servers.xml create mode 100644 tests/integration/test_parallel_replicas_all_marks_read/test.py delete mode 100644 tests/integration/test_parallel_replicas_distributed_read_from_all/configs/remote_servers.xml delete mode 100644 tests/integration/test_parallel_replicas_distributed_read_from_all/test.py delete mode 100644 tests/integration/test_parallel_replicas_working_set/__init__.py delete mode 100644 tests/integration/test_parallel_replicas_working_set/configs/remote_servers.xml delete mode 100644 tests/integration/test_parallel_replicas_working_set/test.py diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 4bdf6288a1c..119e0d99143 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -288,6 +288,18 @@ The server successfully detected this situation and will download merged part fr M(OSReadChars, "Number of bytes read from filesystem, including page cache.") \ M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \ \ + M(ParallelReplicasHandleRequestMicroseconds, "Time spent processing requests for marks from replicas") \ + M(ParallelReplicasHandleAnnouncementMicroseconds, "Time spent processing replicas announcements") \ + \ + M(ParallelReplicasReadAssignedMarks, "Sum across all replicas of how many of scheduled marks were assigned by consistent hash") \ + M(ParallelReplicasReadUnassignedMarks, "Sum across all replicas of how many unassigned marks were scheduled") \ + M(ParallelReplicasReadAssignedForStealingMarks, "Sum across all replicas of how many of scheduled marks were assigned for stealing by consistent hash") \ + \ + M(ParallelReplicasStealingByHashMicroseconds, "Time spent collecting segments meant for stealing by hash") \ + M(ParallelReplicasProcessingPartsMicroseconds, "Time spent processing data parts") \ + M(ParallelReplicasStealingLeftoversMicroseconds, "Time spent collecting orphaned segments") \ + M(ParallelReplicasCollectingOwnedSegmentsMicroseconds, "Time spent collecting segments meant by hash") \ + \ M(PerfCpuCycles, "Total cycles. Be wary of what happens during CPU frequency scaling.") \ M(PerfInstructions, "Retired instructions. Be careful, these can be affected by various issues, most notably hardware interrupt counts.") \ M(PerfCacheReferences, "Cache accesses. Usually, this indicates Last Level Cache accesses, but this may vary depending on your CPU. This may include prefetches and coherency messages; again this depends on the design of your CPU.") \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9516ef72077..4e057861f60 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -185,6 +185,7 @@ class IColumn; M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 0) \ + M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \ \ M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards. Shard is marked as unavailable when: 1) The shard cannot be reached due to a connection failure. 2) Shard is unresolvable through DNS. 3) Table does not exist on the shard.", 0) \ \ diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 18f7280dd19..c448206ed78 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -412,7 +412,8 @@ void executeQueryWithParallelReplicas( new_cluster = not_optimized_cluster->getClusterWithReplicasAsShards(settings, settings.max_parallel_replicas); } - auto coordinator = std::make_shared(new_cluster->getShardCount()); + auto coordinator + = std::make_shared(new_cluster->getShardCount(), settings.parallel_replicas_mark_segment_size); auto external_tables = new_context->getExternalTables(); auto read_from_remote = std::make_unique( query_ast, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index e61ddf0d122..69e64d5ea98 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -1,5 +1,6 @@ #include + namespace DB { @@ -30,12 +31,10 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( settings_, context_) , extension(std::move(extension_)) + , coordination_mode(CoordinationMode::Default) { - extension.all_callback(InitialAllRangesAnnouncement( - CoordinationMode::Default, - parts_ranges.getDescriptions(), - extension.number_of_current_replica - )); + extension.all_callback( + InitialAllRangesAnnouncement(coordination_mode, parts_ranges.getDescriptions(), extension.number_of_current_replica)); } MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t /*task_idx*/, MergeTreeReadTask * previous_task) @@ -48,7 +47,7 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t /*task_id if (buffered_ranges.empty()) { auto result = extension.callback(ParallelReadRequest( - CoordinationMode::Default, + coordination_mode, extension.number_of_current_replica, pool_settings.min_marks_for_concurrent_read * pool_settings.threads, /// For Default coordination mode we don't need to pass part names. diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h index 08020565ec4..7579a892b67 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h @@ -31,6 +31,7 @@ private: mutable std::mutex mutex; const ParallelReadingExtension extension; + const CoordinationMode coordination_mode; RangesInDataPartsDescription buffered_ranges; bool no_more_tasks_available{false}; Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPoolParallelReplicas"); diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 333a0590d6b..bbe8c30a5c0 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -1,27 +1,77 @@ #include #include +#include +#include +#include +#include #include #include -#include -#include #include - +#include +#include +#include +#include #include -#include "Common/Exception.h" -#include -#include -#include -#include -#include "IO/WriteBufferFromString.h" #include -#include "Storages/MergeTree/RangesInDataPart.h" -#include "Storages/MergeTree/RequestResponse.h" -#include +#include #include +#include +#include +#include +#include +#include +#include +#include #include #include +#include +#include +#include +#include +#include +#include + +using namespace DB; + +namespace +{ +size_t roundDownToMultiple(size_t num, size_t multiple) +{ + return (num / multiple) * multiple; +} + +size_t +takeFromRange(const MarkRange & range, size_t min_number_of_marks, size_t & current_marks_amount, RangesInDataPartDescription & result) +{ + const auto marks_needed = min_number_of_marks - current_marks_amount; + chassert(marks_needed); + auto range_we_take = MarkRange{range.begin, range.begin + std::min(marks_needed, range.getNumberOfMarks())}; + if (!result.ranges.empty() && result.ranges.back().end == range_we_take.begin) + /// Can extend the previous range + result.ranges.back().end = range_we_take.end; + else + result.ranges.emplace_back(range_we_take); + current_marks_amount += range_we_take.getNumberOfMarks(); + return range_we_take.getNumberOfMarks(); +} +} + +namespace ProfileEvents +{ +extern const Event ParallelReplicasHandleRequestMicroseconds; +extern const Event ParallelReplicasHandleAnnouncementMicroseconds; + +extern const Event ParallelReplicasStealingByHashMicroseconds; +extern const Event ParallelReplicasProcessingPartsMicroseconds; +extern const Event ParallelReplicasStealingLeftoversMicroseconds; +extern const Event ParallelReplicasCollectingOwnedSegmentsMicroseconds; + +extern const Event ParallelReplicasReadAssignedMarks; +extern const Event ParallelReplicasReadUnassignedMarks; +extern const Event ParallelReplicasReadAssignedForStealingMarks; +} namespace ProfileEvents { @@ -58,7 +108,8 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; +extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; } class ParallelReplicasReadingCoordinator::ImplInterface @@ -68,6 +119,15 @@ public: { size_t number_of_requests{0}; size_t sum_marks{0}; + + /// Marks assigned to the given replica by consistent hash + size_t assigned_to_me = 0; + /// Marks stolen from other replicas + size_t stolen_unassigned = 0; + + /// Stolen marks that were assigned for stealing to the given replica by hash. Makes sense only for DefaultCoordinator + size_t stolen_by_hash = 0; + bool is_unavailable{false}; }; using Stats = std::vector; @@ -76,7 +136,15 @@ public: String result = "Statistics: "; std::vector stats_by_replica; for (size_t i = 0; i < stats.size(); ++i) - stats_by_replica.push_back(fmt::format("replica {}{} - {{requests: {} marks: {}}}", i, stats[i].is_unavailable ? " is unavailable" : "", stats[i].number_of_requests, stats[i].sum_marks)); + stats_by_replica.push_back(fmt::format( + "replica {}{} - {{requests: {} marks: {} assigned_to_me: {} stolen_by_hash: {} stolen_unassigned: {}}}", + i, + stats[i].is_unavailable ? " is unavailable" : "", + stats[i].number_of_requests, + stats[i].sum_marks, + stats[i].assigned_to_me, + stats[i].stolen_by_hash, + stats[i].stolen_unassigned)); result += fmt::format("{}", fmt::join(stats_by_replica, "; ")); return result; } @@ -92,6 +160,7 @@ public: {} virtual ~ImplInterface() = default; + virtual ParallelReadResponse handleRequest(ParallelReadRequest request) = 0; virtual void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) = 0; virtual void markReplicaAsUnavailable(size_t replica_number) = 0; @@ -103,165 +172,227 @@ using Parts = std::set; using PartRefs = std::deque; +/// This coordinator relies heavily on the fact that we work with a single shard, +/// i.e. the difference in parts contained in each replica's snapshot is rather negligible (it is only recently inserted or merged parts). +/// So the guarantees we provide here are basically the same as with single-node reading: we will read from parts as their were seen by some node at the moment when query started. +/// +/// Knowing that almost each part could be read by each node, we suppose ranges of each part to be available to all the replicas and thus distribute them evenly between them +/// (of course we still check if replica has access to the given part before scheduling a reading from it). +/// +/// Of course we want to distribute marks evenly. Looks like it is better to split parts into reasonably small segments of equal size +/// (something between 16 and 128 granules i.e. ~100K and ~1M rows should work). +/// This approach seems to work ok for all three main cases: full scan, reading random sub-ranges and reading only {pre,suf}-fix of parts. +/// Also we could expect that more granular division will make distribution more even up to a certain point. class DefaultCoordinator : public ParallelReplicasReadingCoordinator::ImplInterface { public: - using ParallelReadRequestPtr = std::unique_ptr; - using PartToMarkRanges = std::map; - - explicit DefaultCoordinator(size_t replicas_count_) + explicit DefaultCoordinator(size_t replicas_count_, size_t mark_segment_size_) : ParallelReplicasReadingCoordinator::ImplInterface(replicas_count_) - , reading_state(replicas_count_) + , mark_segment_size(mark_segment_size_) + , replica_status(replicas_count_) + , distribution_by_hash_queue(replicas_count_) { + if (mark_segment_size == 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Zero value provided for `mark_segment_size`"); } ~DefaultCoordinator() override; - struct PartitionReading - { - PartSegments part_ranges; - PartToMarkRanges mark_ranges_in_part; - }; + ParallelReadResponse handleRequest(ParallelReadRequest request) override; - using PartitionToBlockRanges = std::map; - PartitionToBlockRanges partitions; + void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) override; + + void markReplicaAsUnavailable(size_t replica_number) override; + +private: + /// This many granules will represent a single segment of marks that will be assigned to a replica + const size_t mark_segment_size{0}; size_t sent_initial_requests{0}; + bool state_initialized{false}; + size_t finished_replicas{0}; - Parts all_parts_to_read; - /// Contains only parts which we haven't started to read from - PartRefs delayed_parts; - /// Per-replica preferred parts split by consistent hash - /// Once all task will be done by some replica, it can steal tasks - std::vector reading_state; + struct ReplicaStatus + { + bool is_finished{false}; + bool is_announcement_received{false}; + }; + std::vector replica_status; Poco::Logger * log = &Poco::Logger::get("DefaultCoordinator"); - std::atomic state_initialized{false}; + /// Workflow of a segment: + /// 0. `all_parts_to_read` contains all the parts and thus all the segments initially present there (virtually) + /// 1. when we traverse `all_parts_to_read` in selectPartsAndRanges() we either: + /// * take this segment into output + /// * put this segment into `distribution_by_hash_queue` for its owner if it's available and can read from it + /// * otherwise put this segment into `distribution_by_hash_queue` for its stealer_by_hash if it's available and can read from it + /// * otherwise put this segment into `ranges_for_stealing_queue` + /// 2. when we traverse `distribution_by_hash_queue` in `selectPartsAndRanges` we either: + /// * take this segment into output + /// * otherwise put this segment into `distribution_by_hash_queue` for its stealer_by_hash if it's available and can read from it + /// * otherwise put this segment into `ranges_for_stealing_queue` + /// 3. when we figuring out that some replica is unavailable we move all segments from its `distribution_by_hash_queue` to their stealers by hash or to `ranges_for_stealing_queue` + /// 4. when we get the announcement from a replica we move all segments it cannot read to their stealers by hash or to `ranges_for_stealing_queue` + /// + /// So, segments always move in one direction down this path (possibly skipping some stops): + /// `all_parts_to_read` -> `distribution_by_hash_queue[owner]` -> `distribution_by_hash_queue[stealer_by_hash]` -> `ranges_for_stealing_queue` - ParallelReadResponse handleRequest(ParallelReadRequest request) override; - void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) override; - void markReplicaAsUnavailable(size_t replica_number) override; + /// We take the set of parts announced by this replica as the working set for the whole query. + /// For this replica we know for sure that + /// 1. it sees all the parts from this set + /// 2. it was available in the beginning of execution (since we got announcement), so if it will become unavailable at some point - query will be failed with exception. + /// this means that we can delegate reading of all leftover segments (i.e. segments that were not read by their owner or stealer by hash) to this node + size_t source_replica_for_parts_snapshot{0}; - void updateReadingState(InitialAllRangesAnnouncement announcement); - void finalizeReadingState(); + /// Parts view from the first announcement we received + std::vector all_parts_to_read; - size_t computeConsistentHash(const MergeTreePartInfo & info) const + std::unordered_map> part_visibility; /// part_name -> set of replicas announced that part + + /// We order parts from biggest (= oldest) to newest and steal from newest. Because we assume + /// that they're gonna be merged soon anyway and for them we should already expect worse cache hit. + struct BiggerPartsFirst { - auto hash = SipHash(); - hash.update(info.getPartNameV1()); - return ConsistentHashing(hash.get64(), replicas_count); - } + bool operator()(const auto & lhs, const auto & rhs) const { return lhs.info.getBlocksCount() > rhs.info.getBlocksCount(); } + }; - void selectPartsAndRanges(const PartRefs & container, size_t replica_num, size_t min_number_of_marks, size_t & current_mark_size, ParallelReadResponse & response) const; + /// We don't precalculate the whole assignment for each node at the start. + /// When replica asks coordinator for a new portion of data to read, it traverses `all_parts_to_read` to find ranges relevant to this replica (by consistent hash). + /// Many hashes are being calculated during this process and just to not loose this time we save the information about all these ranges + /// observed along the way to what node they belong to. + /// Ranges in this queue might belong to a part that the given replica cannot read from - the corresponding check happens later. + /// TODO: consider making it bounded in size + std::vector> distribution_by_hash_queue; + + /// For some ranges their owner and stealer (by consistent hash) cannot read from the given part at all. So this range have to be stolen anyway. + /// TODO: consider making it bounded in size + RangesInDataPartsDescription ranges_for_stealing_queue; + + /// We take only first replica's set of parts as the whole working set for this query. + /// For other replicas we'll just discard parts that they know, but that weren't present in the first request we received. + /// The second and all subsequent announcements needed only to understand if we can schedule reading from the given part to the given replica. + void initializeReadingState(InitialAllRangesAnnouncement announcement); + + void setProgressCallback(); + + enum class ScanMode + { + /// Main working set for the replica + TakeWhatsMineByHash, + /// We need to steal to optimize tail latency, let's do it by hash nevertheless + TakeWhatsMineForStealing, + /// All bets are off, we need to steal "for correctness" - to not leave any segments unread + TakeEverythingAvailable + }; + + void selectPartsAndRanges( + size_t replica_num, + ScanMode scan_mode, + size_t min_number_of_marks, + size_t & current_marks_amount, + RangesInDataPartsDescription & description); + + size_t computeConsistentHash(const std::string & part_name, size_t segment_begin, ScanMode scan_mode) const; + + void tryToTakeFromDistributionQueue( + size_t replica_num, size_t min_number_of_marks, size_t & current_marks_amount, RangesInDataPartsDescription & description); + + void tryToStealFromQueues( + size_t replica_num, + ScanMode scan_mode, + size_t min_number_of_marks, + size_t & current_marks_amount, + RangesInDataPartsDescription & description); + + void tryToStealFromQueue( + auto & queue, + ssize_t owner, /// In case `queue` is `distribution_by_hash_queue[replica]` + size_t replica_num, + ScanMode scan_mode, + size_t min_number_of_marks, + size_t & current_marks_amount, + RangesInDataPartsDescription & description); + + void processPartsFurther( + size_t replica_num, + ScanMode scan_mode, + size_t min_number_of_marks, + size_t & current_marks_amount, + RangesInDataPartsDescription & description); + + bool possiblyCanReadPart(size_t replica, const MergeTreePartInfo & info) const; + void enqueueSegment(const MergeTreePartInfo & info, const MarkRange & segment, size_t owner); + void enqueueToStealerOrStealingQueue(const MergeTreePartInfo & info, const MarkRange & segment); }; + DefaultCoordinator::~DefaultCoordinator() { - LOG_DEBUG(log, "Coordination done: {}", toString(stats)); + try + { + LOG_DEBUG(log, "Coordination done: {}", toString(stats)); + } + catch (...) + { + tryLogCurrentException(log); + } } -void DefaultCoordinator::updateReadingState(InitialAllRangesAnnouncement announcement) +void DefaultCoordinator::initializeReadingState(InitialAllRangesAnnouncement announcement) { - PartRefs parts_diff; - - /// To get rid of duplicates - for (auto && part_ranges: announcement.description) + for (const auto & part : announcement.description) { - Part part{.description = std::move(part_ranges), .replicas = {announcement.replica_num}}; - const MergeTreePartInfo & announced_part = part.description.info; - - auto it = std::lower_bound(cbegin(all_parts_to_read), cend(all_parts_to_read), part); - if (it != all_parts_to_read.cend()) - { - const MergeTreePartInfo & found_part = it->description.info; - if (found_part == announced_part) - { - /// We have the same part - add the info about presence on current replica - it->replicas.insert(announcement.replica_num); - continue; - } - else - { - /// check if it is covering or covered part - /// need to compare with 2 nearest parts in set, - lesser and greater than the part from the announcement - bool is_disjoint = found_part.isDisjoint(announced_part); - if (it != all_parts_to_read.cbegin() && is_disjoint) - { - const MergeTreePartInfo & lesser_part = (--it)->description.info; - is_disjoint &= lesser_part.isDisjoint(announced_part); - } - if (!is_disjoint) - continue; - } - } - else if (!all_parts_to_read.empty()) - { - /// the announced part is greatest - check if it's disjoint with lesser part - const MergeTreePartInfo & lesser_part = all_parts_to_read.crbegin()->description.info; - if (!lesser_part.isDisjoint(announced_part)) - continue; - } - - auto [insert_it, _] = all_parts_to_read.emplace(std::move(part)); - parts_diff.push_back(insert_it); + /// We don't really care here if this part will be included into the working set or not + part_visibility[part.info.getPartNameV1()].insert(announcement.replica_num); } - /// Split all parts by consistent hash - while (!parts_diff.empty()) + /// If state is already initialized - just register availabitily info and leave + if (state_initialized) + return; + + for (auto && part : announcement.description) { - auto current_part_it = parts_diff.front(); - parts_diff.pop_front(); - auto consistent_hash = computeConsistentHash(current_part_it->description.info); + auto intersecting_it = std::find_if( + all_parts_to_read.begin(), + all_parts_to_read.end(), + [&part](const Part & other) { return !other.description.info.isDisjoint(part.info); }); - /// Check whether the new part can easy go to replica queue - if (current_part_it->replicas.contains(consistent_hash)) - { - reading_state[consistent_hash].emplace_back(current_part_it); - continue; - } + if (intersecting_it != all_parts_to_read.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Intersecting parts found in announcement"); - /// Add to delayed parts - delayed_parts.emplace_back(current_part_it); + all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); } + + std::ranges::sort( + all_parts_to_read, [](const Part & lhs, const Part & rhs) { return BiggerPartsFirst()(lhs.description, rhs.description); }); + state_initialized = true; + source_replica_for_parts_snapshot = announcement.replica_num; + + LOG_DEBUG(log, "Reading state is fully initialized: {}", fmt::join(all_parts_to_read, "; ")); } void DefaultCoordinator::markReplicaAsUnavailable(size_t replica_number) { - if (stats[replica_number].is_unavailable == false) + LOG_DEBUG(log, "Replica number {} is unavailable", replica_number); + + ++unavailable_replicas_count; + stats[replica_number].is_unavailable = true; + + if (sent_initial_requests == replicas_count - unavailable_replicas_count) + setProgressCallback(); + + for (const auto & segment : distribution_by_hash_queue[replica_number]) { - LOG_DEBUG(log, "Replica number {} is unavailable", replica_number); - - stats[replica_number].is_unavailable = true; - ++unavailable_replicas_count; - - if (sent_initial_requests == replicas_count - unavailable_replicas_count) - finalizeReadingState(); + chassert(segment.ranges.size() == 1); + enqueueToStealerOrStealingQueue(segment.info, segment.ranges.front()); } + distribution_by_hash_queue[replica_number].clear(); } -void DefaultCoordinator::finalizeReadingState() +void DefaultCoordinator::setProgressCallback() { - /// Clear all the delayed queue - while (!delayed_parts.empty()) - { - auto current_part_it = delayed_parts.front(); - auto consistent_hash = computeConsistentHash(current_part_it->description.info); - - if (current_part_it->replicas.contains(consistent_hash)) - { - reading_state[consistent_hash].emplace_back(current_part_it); - delayed_parts.pop_front(); - continue; - } - - /// In this situation just assign to a random replica which has this part - auto replica = *(std::next(current_part_it->replicas.begin(), thread_local_rng() % current_part_it->replicas.size())); - reading_state[replica].emplace_back(current_part_it); - delayed_parts.pop_front(); - } - - // update progress with total rows + // Update progress with total rows if (progress_callback) { size_t total_rows_to_read = 0; @@ -274,116 +405,378 @@ void DefaultCoordinator::finalizeReadingState() LOG_DEBUG(log, "Total rows to read: {}", total_rows_to_read); } - - LOG_DEBUG(log, "Reading state is fully initialized: {}", fmt::join(all_parts_to_read, "; ")); } - void DefaultCoordinator::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { const auto replica_num = announcement.replica_num; - updateReadingState(std::move(announcement)); + LOG_DEBUG(log, "Initial request from replica {}: {}", announcement.replica_num, announcement.describe()); + + initializeReadingState(std::move(announcement)); if (replica_num >= stats.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Replica number ({}) is bigger than total replicas count ({})", replica_num, stats.size()); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Replica number ({}) is bigger than total replicas count ({})", replica_num, stats.size()); ++stats[replica_num].number_of_requests; + replica_status[replica_num].is_announcement_received = true; ++sent_initial_requests; LOG_DEBUG(log, "Sent initial requests: {} Replicas count: {}", sent_initial_requests, replicas_count); + if (sent_initial_requests == replicas_count) - finalizeReadingState(); -} + setProgressCallback(); -void DefaultCoordinator::selectPartsAndRanges(const PartRefs & container, size_t replica_num, size_t min_number_of_marks, size_t & current_mark_size, ParallelReadResponse & response) const -{ - for (const auto & part : container) + /// Sift the queue to move out all invisible segments + for (const auto & segment : distribution_by_hash_queue[replica_num]) { - if (current_mark_size >= min_number_of_marks) + if (!part_visibility[segment.info.getPartNameV1()].contains(replica_num)) { - LOG_TEST(log, "Current mark size {} is bigger than min_number_marks {}", current_mark_size, min_number_of_marks); - break; - } - - if (part->description.ranges.empty()) - { - LOG_TEST(log, "Part {} is already empty in reading state", part->description.info.getPartNameV1()); - continue; - } - - if (std::find(part->replicas.begin(), part->replicas.end(), replica_num) == part->replicas.end()) - { - LOG_TEST(log, "Not found part {} on replica {}", part->description.info.getPartNameV1(), replica_num); - continue; - } - - response.description.push_back({ - .info = part->description.info, - .ranges = {}, - }); - - while (!part->description.ranges.empty() && current_mark_size < min_number_of_marks) - { - auto & range = part->description.ranges.front(); - const size_t needed = min_number_of_marks - current_mark_size; - - if (range.getNumberOfMarks() > needed) - { - auto range_we_take = MarkRange{range.begin, range.begin + needed}; - response.description.back().ranges.emplace_back(range_we_take); - current_mark_size += range_we_take.getNumberOfMarks(); - - range.begin += needed; - break; - } - - response.description.back().ranges.emplace_back(range); - current_mark_size += range.getNumberOfMarks(); - part->description.ranges.pop_front(); + chassert(segment.ranges.size() == 1); + enqueueToStealerOrStealingQueue(segment.info, segment.ranges.front()); } } } +void DefaultCoordinator::tryToTakeFromDistributionQueue( + size_t replica_num, size_t min_number_of_marks, size_t & current_marks_amount, RangesInDataPartsDescription & description) +{ + ProfileEventTimeIncrement watch(ProfileEvents::ParallelReplicasCollectingOwnedSegmentsMicroseconds); + + auto & distribution_queue = distribution_by_hash_queue[replica_num]; + auto replica_can_read_part = [&](auto replica, const auto & part) { return part_visibility[part.getPartNameV1()].contains(replica); }; + + RangesInDataPartDescription result; + + while (!distribution_queue.empty() && current_marks_amount < min_number_of_marks) + { + if (result.ranges.empty() || distribution_queue.begin()->info != result.info) + { + if (!result.ranges.empty()) + /// We're switching to a different part, so have to save currently accumulated ranges + description.push_back(result); + result = {.info = distribution_queue.begin()->info}; + } + + /// NOTE: this works because ranges are not considered by the comparator + auto & part_ranges = const_cast(*distribution_queue.begin()); + chassert(part_ranges.ranges.size() == 1); + auto & range = part_ranges.ranges.front(); + + if (replica_can_read_part(replica_num, part_ranges.info)) + { + if (auto taken = takeFromRange(range, min_number_of_marks, current_marks_amount, result); taken == range.getNumberOfMarks()) + distribution_queue.erase(distribution_queue.begin()); + else + { + range.begin += taken; + break; + } + } + else + { + /// It might be that `replica_num` is the stealer by hash itself - no problem, + /// we'll just have a redundant hash computation inside this function + enqueueToStealerOrStealingQueue(part_ranges.info, range); + distribution_queue.erase(distribution_queue.begin()); + } + } + + if (!result.ranges.empty()) + description.push_back(result); +} + +void DefaultCoordinator::tryToStealFromQueues( + size_t replica_num, + ScanMode scan_mode, + size_t min_number_of_marks, + size_t & current_marks_amount, + RangesInDataPartsDescription & description) +{ + auto steal_from_other_replicas = [&]() + { + /// Try to steal from other replicas starting from replicas with longest queues + std::vector order(replicas_count); + std::iota(order.begin(), order.end(), 0); + std::ranges::sort( + order, [&](auto lhs, auto rhs) { return distribution_by_hash_queue[lhs].size() > distribution_by_hash_queue[rhs].size(); }); + + for (auto replica : order) + tryToStealFromQueue( + distribution_by_hash_queue[replica], + replica, + replica_num, + scan_mode, + min_number_of_marks, + current_marks_amount, + description); + }; + + if (scan_mode == ScanMode::TakeWhatsMineForStealing) + { + ProfileEventTimeIncrement watch(ProfileEvents::ParallelReplicasStealingByHashMicroseconds); + steal_from_other_replicas(); + } + else + { + ProfileEventTimeIncrement watch(ProfileEvents::ParallelReplicasStealingLeftoversMicroseconds); + /// Check orphaned ranges + tryToStealFromQueue( + ranges_for_stealing_queue, /*owner=*/-1, replica_num, scan_mode, min_number_of_marks, current_marks_amount, description); + /// Last hope. In case we haven't yet figured out that some node is unavailable its segments are still in the distribution queue. + steal_from_other_replicas(); + } +} + +void DefaultCoordinator::tryToStealFromQueue( + auto & queue, + ssize_t owner, + size_t replica_num, + ScanMode scan_mode, + size_t min_number_of_marks, + size_t & current_marks_amount, + RangesInDataPartsDescription & description) +{ + auto replica_can_read_part = [&](auto replica, const auto & part) { return part_visibility[part.getPartNameV1()].contains(replica); }; + + RangesInDataPartDescription result; + + auto it = queue.rbegin(); + while (it != queue.rend() && current_marks_amount < min_number_of_marks) + { + auto & part_ranges = const_cast(*it); + chassert(part_ranges.ranges.size() == 1); + auto & range = part_ranges.ranges.front(); + + if (result.ranges.empty() || part_ranges.info != result.info) + { + if (!result.ranges.empty()) + /// We're switching to a different part, so have to save currently accumulated ranges + description.push_back(result); + result = {.info = part_ranges.info}; + } + + if (replica_can_read_part(replica_num, part_ranges.info)) + { + bool can_take = false; + if (scan_mode == ScanMode::TakeWhatsMineForStealing) + { + chassert(owner >= 0); + const size_t segment_begin = roundDownToMultiple(range.begin, mark_segment_size); + can_take = computeConsistentHash(part_ranges.info.getPartNameV1(), segment_begin, scan_mode) == replica_num; + } + else + { + /// Don't steal segments with alive owner that sees them + can_take = owner == -1 || stats[owner].is_unavailable || !replica_status[owner].is_announcement_received; + } + if (can_take) + { + if (auto taken = takeFromRange(range, min_number_of_marks, current_marks_amount, result); taken == range.getNumberOfMarks()) + { + it = decltype(it)(queue.erase(std::next(it).base())); + continue; + } + else + range.begin += taken; + } + } + + ++it; + } + + if (!result.ranges.empty()) + description.push_back(result); +} + +void DefaultCoordinator::processPartsFurther( + size_t replica_num, + ScanMode scan_mode, + size_t min_number_of_marks, + size_t & current_marks_amount, + RangesInDataPartsDescription & description) +{ + ProfileEventTimeIncrement watch(ProfileEvents::ParallelReplicasProcessingPartsMicroseconds); + + for (const auto & part : all_parts_to_read) + { + if (current_marks_amount >= min_number_of_marks) + { + LOG_TEST(log, "Current mark size {} is bigger than min_number_marks {}", current_marks_amount, min_number_of_marks); + return; + } + + RangesInDataPartDescription result{.info = part.description.info}; + + while (!part.description.ranges.empty() && current_marks_amount < min_number_of_marks) + { + auto & range = part.description.ranges.front(); + + /// Parts are divided into segments of `mark_segment_size` granules staring from 0-th granule + for (size_t segment_begin = roundDownToMultiple(range.begin, mark_segment_size); + segment_begin < range.end && current_marks_amount < min_number_of_marks; + segment_begin += mark_segment_size) + { + const auto cur_segment + = MarkRange{std::max(range.begin, segment_begin), std::min(range.end, segment_begin + mark_segment_size)}; + + const auto owner = computeConsistentHash(part.description.info.getPartNameV1(), segment_begin, scan_mode); + if (owner == replica_num) + { + const auto taken = takeFromRange(cur_segment, min_number_of_marks, current_marks_amount, result); + if (taken == range.getNumberOfMarks()) + part.description.ranges.pop_front(); + else + { + range.begin += taken; + break; + } + } + else + { + chassert(scan_mode == ScanMode::TakeWhatsMineByHash); + enqueueSegment(part.description.info, cur_segment, owner); + range.begin += cur_segment.getNumberOfMarks(); + if (range.getNumberOfMarks() == 0) + part.description.ranges.pop_front(); + } + } + } + + if (!result.ranges.empty()) + description.push_back(std::move(result)); + } +} + +void DefaultCoordinator::selectPartsAndRanges( + size_t replica_num, + ScanMode scan_mode, + size_t min_number_of_marks, + size_t & current_marks_amount, + RangesInDataPartsDescription & description) +{ + if (scan_mode == ScanMode::TakeWhatsMineByHash) + { + tryToTakeFromDistributionQueue(replica_num, min_number_of_marks, current_marks_amount, description); + processPartsFurther(replica_num, scan_mode, min_number_of_marks, current_marks_amount, description); + /// We might back-fill `distribution_by_hash_queue` for this replica in `enqueueToStealerOrStealingQueue` + tryToTakeFromDistributionQueue(replica_num, min_number_of_marks, current_marks_amount, description); + } + else + tryToStealFromQueues(replica_num, scan_mode, min_number_of_marks, current_marks_amount, description); +} + +bool DefaultCoordinator::possiblyCanReadPart(size_t replica, const MergeTreePartInfo & info) const +{ + /// At this point we might not be sure if `owner` can read from the given part. + /// Then we will check it while processing `owner`'s data requests - they are guaranteed to came after the announcement. + return !stats[replica].is_unavailable && !replica_status[replica].is_finished + && (!replica_status[replica].is_announcement_received || part_visibility.at(info.getPartNameV1()).contains(replica)); +} + +void DefaultCoordinator::enqueueSegment(const MergeTreePartInfo & info, const MarkRange & segment, size_t owner) +{ + if (possiblyCanReadPart(owner, info)) + { + /// TODO: optimize me (maybe we can store something lighter than RangesInDataPartDescription) + distribution_by_hash_queue[owner].insert(RangesInDataPartDescription{.info = info, .ranges = {segment}}); + LOG_TEST(log, "Segment {} is added to its owner's ({}) queue", segment, owner); + } + else + enqueueToStealerOrStealingQueue(info, segment); +} + +void DefaultCoordinator::enqueueToStealerOrStealingQueue(const MergeTreePartInfo & info, const MarkRange & segment) +{ + auto && range = RangesInDataPartDescription{.info = info, .ranges = {segment}}; + const auto stealer_by_hash = computeConsistentHash( + info.getPartNameV1(), roundDownToMultiple(segment.begin, mark_segment_size), ScanMode::TakeWhatsMineForStealing); + if (possiblyCanReadPart(stealer_by_hash, info)) + { + distribution_by_hash_queue[stealer_by_hash].insert(std::move(range)); + LOG_TEST(log, "Segment {} is added to its stealer's ({}) queue", segment, stealer_by_hash); + } + else + { + ranges_for_stealing_queue.push_back(std::move(range)); + LOG_TEST(log, "Segment {} is added to stealing queue", segment); + } +} + +size_t DefaultCoordinator::computeConsistentHash(const std::string & part_name, size_t segment_begin, ScanMode scan_mode) const +{ + chassert(segment_begin % mark_segment_size == 0); + auto hash = SipHash(); + hash.update(part_name); + hash.update(segment_begin); + hash.update(scan_mode); + return ConsistentHashing(hash.get64(), replicas_count); +} + ParallelReadResponse DefaultCoordinator::handleRequest(ParallelReadRequest request) { LOG_TRACE(log, "Handling request from replica {}, minimal marks size is {}", request.replica_num, request.min_number_of_marks); - size_t current_mark_size = 0; ParallelReadResponse response; - /// 1. Try to select from preferred set of parts for current replica - selectPartsAndRanges(reading_state[request.replica_num], request.replica_num, request.min_number_of_marks, current_mark_size, response); + size_t current_mark_size = 0; - /// 2. Try to use parts from delayed queue - while (!delayed_parts.empty() && current_mark_size < request.min_number_of_marks) - { - auto part = delayed_parts.front(); - delayed_parts.pop_front(); - reading_state[request.replica_num].emplace_back(part); - selectPartsAndRanges(reading_state[request.replica_num], request.replica_num, request.min_number_of_marks, current_mark_size, response); - } + /// 1. Try to select ranges meant for this replica by consistent hash + selectPartsAndRanges( + request.replica_num, ScanMode::TakeWhatsMineByHash, request.min_number_of_marks, current_mark_size, response.description); + const size_t assigned_to_me = current_mark_size; - /// 3. Try to steal tasks; - if (current_mark_size < request.min_number_of_marks) - { - for (size_t i = 0; i < replicas_count; ++i) - { - if (i != request.replica_num) - selectPartsAndRanges(reading_state[i], request.replica_num, request.min_number_of_marks, current_mark_size, response); + /// 2. Try to steal but with caching again (with different key) + selectPartsAndRanges( + request.replica_num, ScanMode::TakeWhatsMineForStealing, request.min_number_of_marks, current_mark_size, response.description); + const size_t stolen_by_hash = current_mark_size - assigned_to_me; - if (current_mark_size >= request.min_number_of_marks) - break; - } - } + /// 3. Try to steal with no preference. We're trying to postpone it as much as possible. + if (current_mark_size == 0 && request.replica_num == source_replica_for_parts_snapshot) + selectPartsAndRanges( + request.replica_num, ScanMode::TakeEverythingAvailable, request.min_number_of_marks, current_mark_size, response.description); + const size_t stolen_unassigned = current_mark_size - stolen_by_hash - assigned_to_me; stats[request.replica_num].number_of_requests += 1; stats[request.replica_num].sum_marks += current_mark_size; + stats[request.replica_num].assigned_to_me += assigned_to_me; + stats[request.replica_num].stolen_by_hash += stolen_by_hash; + stats[request.replica_num].stolen_unassigned += stolen_unassigned; + + ProfileEvents::increment(ProfileEvents::ParallelReplicasReadAssignedMarks, assigned_to_me); + ProfileEvents::increment(ProfileEvents::ParallelReplicasReadUnassignedMarks, stolen_unassigned); + ProfileEvents::increment(ProfileEvents::ParallelReplicasReadAssignedForStealingMarks, stolen_by_hash); + if (response.description.empty()) + { response.finish = true; - LOG_TRACE(log, "Going to respond to replica {} with {}", request.replica_num, response.describe()); + replica_status[request.replica_num].is_finished = true; + + if (++finished_replicas == replicas_count - unavailable_replicas_count) + { + /// Nobody will come to process any more data + + if (!ranges_for_stealing_queue.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Some orphaned segments were left unread"); + + for (size_t replica = 0; replica < replicas_count; ++replica) + if (!distribution_by_hash_queue[replica].empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Non-empty distribution_by_hash_queue for replica {}", replica); + } + } + + LOG_DEBUG( + log, + "Going to respond to replica {} with {}; mine_marks={}, stolen_by_hash={}, stolen_rest={}", + request.replica_num, + response.describe(), + assigned_to_me, + stolen_by_hash, + stolen_unassigned); + return response; } @@ -456,6 +849,8 @@ void InOrderCoordinator::handleInitialAllRangesAnnouncement(InitialAllRang std::sort(ranges.begin(), ranges.end()); } + ++stats[announcement.replica_num].number_of_requests; + if (new_rows_to_read > 0) { Progress progress; @@ -557,6 +952,8 @@ ParallelReadResponse InOrderCoordinator::handleRequest(ParallelReadRequest void ParallelReplicasReadingCoordinator::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { + ProfileEventTimeIncrement watch(ProfileEvents::ParallelReplicasHandleAnnouncementMicroseconds); + std::lock_guard lock(mutex); if (!pimpl) @@ -570,6 +967,8 @@ void ParallelReplicasReadingCoordinator::handleInitialAllRangesAnnouncement(Init ParallelReadResponse ParallelReplicasReadingCoordinator::handleRequest(ParallelReadRequest request) { + ProfileEventTimeIncrement watch(ProfileEvents::ParallelReplicasHandleRequestMicroseconds); + std::lock_guard lock(mutex); if (!pimpl) @@ -604,7 +1003,7 @@ void ParallelReplicasReadingCoordinator::initialize() switch (mode) { case CoordinationMode::Default: - pimpl = std::make_unique(replicas_count); + pimpl = std::make_unique(replicas_count, mark_segment_size); break; case CoordinationMode::WithOrder: pimpl = std::make_unique>(replicas_count); @@ -621,7 +1020,10 @@ void ParallelReplicasReadingCoordinator::initialize() pimpl->markReplicaAsUnavailable(replica); } -ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator(size_t replicas_count_) : replicas_count(replicas_count_) {} +ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator(size_t replicas_count_, size_t mark_segment_size_) + : replicas_count(replicas_count_), mark_segment_size(mark_segment_size_) +{ +} ParallelReplicasReadingCoordinator::~ParallelReplicasReadingCoordinator() = default; diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h index acc265c124f..9cba7d8e8c2 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h @@ -15,7 +15,7 @@ class ParallelReplicasReadingCoordinator public: class ImplInterface; - explicit ParallelReplicasReadingCoordinator(size_t replicas_count_); + explicit ParallelReplicasReadingCoordinator(size_t replicas_count_, size_t mark_segment_size_ = 0); ~ParallelReplicasReadingCoordinator(); void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement); @@ -35,8 +35,8 @@ private: std::mutex mutex; size_t replicas_count{0}; + size_t mark_segment_size{0}; CoordinationMode mode{CoordinationMode::Default}; - std::atomic initialized{false}; std::unique_ptr pimpl; ProgressCallback progress_callback; // store the callback only to bypass it to coordinator implementation std::set replicas_used; diff --git a/tests/integration/test_parallel_replicas_distributed_read_from_all/__init__.py b/tests/integration/test_parallel_replicas_all_marks_read/__init__.py similarity index 100% rename from tests/integration/test_parallel_replicas_distributed_read_from_all/__init__.py rename to tests/integration/test_parallel_replicas_all_marks_read/__init__.py diff --git a/tests/integration/test_parallel_replicas_all_marks_read/configs/remote_servers.xml b/tests/integration/test_parallel_replicas_all_marks_read/configs/remote_servers.xml new file mode 100644 index 00000000000..1ad562334f5 --- /dev/null +++ b/tests/integration/test_parallel_replicas_all_marks_read/configs/remote_servers.xml @@ -0,0 +1,32 @@ + + + + + + node0 + 9000 + + + node1 + 9000 + + + node2 + 9000 + + + node3 + 9000 + + + node4 + 9000 + + + node5 + 9000 + + + + + diff --git a/tests/integration/test_parallel_replicas_all_marks_read/test.py b/tests/integration/test_parallel_replicas_all_marks_read/test.py new file mode 100644 index 00000000000..7776ccb0c09 --- /dev/null +++ b/tests/integration/test_parallel_replicas_all_marks_read/test.py @@ -0,0 +1,156 @@ +import json +import pytest + +from helpers.cluster import ClickHouseCluster +from random import randint + +cluster = ClickHouseCluster(__file__) +cluster_name = "parallel_replicas_with_unavailable_nodes" + +nodes = [ + cluster.add_instance( + f"node{num}", main_configs=["configs/remote_servers.xml"], with_zookeeper=True + ) + for num in range(3) +] + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def _create_tables(table_name, table_size, index_granularity): + for num in range(len(nodes)): + nodes[num].query(f"DROP TABLE IF EXISTS {table_name}") + + nodes[num].query( + f""" + CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) + Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', '{num}') + ORDER BY (key) + SETTINGS index_granularity = {index_granularity} + """ + ) + + nodes[0].query( + f""" + INSERT INTO {table_name} + SELECT number, toString(number) FROM numbers_mt({table_size}) + """ + ) + + +def _create_query(query_tmpl, table_name): + rand_set = [randint(0, 500) for i in range(42)] + return query_tmpl.format(table_name=table_name, rand_set=rand_set) + + +def _get_result_without_parallel_replicas(query): + return nodes[0].query( + query, + settings={ + "allow_experimental_parallel_reading_from_replicas": 0, + }, + ) + + +def _get_result_with_parallel_replicas( + query, query_id, cluster_name, parallel_replicas_mark_segment_size +): + return nodes[0].query( + query, + settings={ + "allow_experimental_parallel_reading_from_replicas": 2, + "max_parallel_replicas": 6, + "cluster_for_parallel_replicas": f"{cluster_name}", + "parallel_replicas_mark_segment_size": parallel_replicas_mark_segment_size, + "query_id": query_id, + }, + ) + + +def _get_expected_amount_of_marks_to_read(query): + return json.loads( + nodes[0].query( + f""" + EXPLAIN ESTIMATE + {query} + FORMAT JSONEachRow + """ + ) + )["marks"] + + +def _get_number_of_marks_read_by_replicas(query_id): + nodes[0].query("SYSTEM FLUSH LOGS") + return ( + nodes[0] + .query( + f""" + SELECT sum( + ProfileEvents['ParallelReplicasReadAssignedMarks'] + + ProfileEvents['ParallelReplicasReadUnassignedMarks'] + + ProfileEvents['ParallelReplicasReadAssignedForStealingMarks'] + ) + FROM system.query_log + WHERE query_id = '{query_id}' + """ + ) + .strip() + ) + + +@pytest.mark.parametrize( + "query_tmpl", + [ + "SELECT sum(cityHash64(*)) FROM {table_name}", + "SELECT sum(cityHash64(*)) FROM {table_name} WHERE intDiv(key, 100) IN {rand_set}", + ], +) +@pytest.mark.parametrize( + "table_size", + [1000, 10000, 100000], +) +@pytest.mark.parametrize( + "index_granularity", + [10, 100], +) +@pytest.mark.parametrize( + "parallel_replicas_mark_segment_size", + [1, 10], +) +def test_number_of_marks_read( + start_cluster, + query_tmpl, + table_size, + index_granularity, + parallel_replicas_mark_segment_size, +): + if nodes[0].is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers (too slow)") + + table_name = f"tbl_{len(query_tmpl)}_{cluster_name}_{table_size}_{index_granularity}_{parallel_replicas_mark_segment_size}" + _create_tables(table_name, table_size, index_granularity) + + if "where" in query_tmpl.lower(): + # We need all the replicas to see the same state of parts to make sure that index analysis will pick the same amount of marks for reading + # regardless of which replica's state will be chosen as the working set. This should became redundant once we start to always use initiator's snapshot. + nodes[0].query(f"OPTIMIZE TABLE {table_name} FINAL", settings={"alter_sync": 2}) + for node in nodes: + node.query(f"SYSTEM SYNC REPLICA {table_name} STRICT") + + query = _create_query(query_tmpl, table_name) + query_id = f"{table_name}_{randint(0, 1e9)}" + + assert _get_result_with_parallel_replicas( + query, query_id, cluster_name, parallel_replicas_mark_segment_size + ) == _get_result_without_parallel_replicas(query) + + assert _get_number_of_marks_read_by_replicas( + query_id + ) == _get_expected_amount_of_marks_to_read(query) diff --git a/tests/integration/test_parallel_replicas_distributed_read_from_all/configs/remote_servers.xml b/tests/integration/test_parallel_replicas_distributed_read_from_all/configs/remote_servers.xml deleted file mode 100644 index 02a315479f8..00000000000 --- a/tests/integration/test_parallel_replicas_distributed_read_from_all/configs/remote_servers.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - - - true - - n1 - 9000 - - - n2 - 9000 - - - n3 - 9000 - - - - - - diff --git a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py deleted file mode 100644 index 8af7bb12595..00000000000 --- a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py +++ /dev/null @@ -1,156 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -nodes = [ - cluster.add_instance( - f"n{i}", main_configs=["configs/remote_servers.xml"], with_zookeeper=True - ) - for i in (1, 2, 3) -] - - -@pytest.fixture(scope="module", autouse=True) -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def create_tables(cluster, table_name): - """create replicated tables in special way - - each table is populated by equal number of rows - - fetches are disabled, so each replica will have different set of rows - which enforce parallel replicas read from each replica - """ - - # create replicated tables - for node in nodes: - node.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - - nodes[0].query( - f"""CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') - ORDER BY (key)""" - ) - nodes[1].query( - f"""CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') - ORDER BY (key)""" - ) - nodes[2].query( - f"""CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') - ORDER BY (key)""" - ) - # stop merges - nodes[0].query(f"system stop merges {table_name}") - nodes[1].query(f"system stop merges {table_name}") - nodes[2].query(f"system stop merges {table_name}") - # stop fetches - nodes[0].query(f"system stop fetches {table_name}") - nodes[1].query(f"system stop fetches {table_name}") - nodes[2].query(f"system stop fetches {table_name}") - - # create distributed table - nodes[0].query(f"DROP TABLE IF EXISTS {table_name}_d SYNC") - nodes[0].query( - f""" - CREATE TABLE {table_name}_d AS {table_name} - Engine=Distributed( - {cluster}, - currentDatabase(), - {table_name}, - rand() - ) - """ - ) - - # populate data, equal number of rows for each replica - nodes[0].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(10)", - settings={"distributed_foreground_insert": 1}, - ) - nodes[0].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(10, 10)", - settings={"distributed_foreground_insert": 1}, - ) - nodes[1].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(20, 10)", - settings={"distributed_foreground_insert": 1}, - ) - nodes[1].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(30, 10)", - settings={"distributed_foreground_insert": 1}, - ) - nodes[2].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(40, 10)", - settings={"distributed_foreground_insert": 1}, - ) - nodes[2].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(50, 10)", - settings={"distributed_foreground_insert": 1}, - ) - - return "60\t0\t59\t1770\n" - - -@pytest.mark.parametrize( - "prefer_localhost_replica", - [ - pytest.param(0), - pytest.param(1), - ], -) -def test_read_equally_from_each_replica(start_cluster, prefer_localhost_replica): - """create and populate table in special way (see create_table()), - so parallel replicas will read equal number of rows from each replica - """ - - cluster = "test_single_shard_multiple_replicas" - table_name = "test_table" - expected_result = create_tables(cluster, table_name) - - # parallel replicas - assert ( - nodes[0].query( - f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", - settings={ - "allow_experimental_parallel_reading_from_replicas": 2, - "prefer_localhost_replica": prefer_localhost_replica, - "max_parallel_replicas": 3, - }, - ) - == expected_result - ) - - # check logs for coordinator statistic - for n in nodes: - n.query("SYSTEM FLUSH LOGS") - - # each replica has 2 distinct parts (non-intersecting with another replicas), - # each part less then index granularity, therefore 2 marks for each replica to handle - coordinator_statistic = "replica 0 - {requests: 3 marks: 2}; replica 1 - {requests: 3 marks: 2}; replica 2 - {requests: 3 marks: 2}" - assert ( - nodes[0].contains_in_log(coordinator_statistic) - or nodes[1].contains_in_log(coordinator_statistic) - or nodes[2].contains_in_log(coordinator_statistic) - ) - - # w/o parallel replicas - # start fetches back, otherwise the result will be not as expected - nodes[0].query(f"system start fetches {table_name}") - nodes[1].query(f"system start fetches {table_name}") - nodes[2].query(f"system start fetches {table_name}") - # ensure that replica in sync before querying it to get stable result - nodes[0].query(f"system start merges {table_name}") - nodes[0].query(f"system sync replica {table_name}") - assert ( - nodes[0].query( - f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", - settings={ - "allow_experimental_parallel_reading_from_replicas": 0, - }, - ) - == expected_result - ) diff --git a/tests/integration/test_parallel_replicas_working_set/__init__.py b/tests/integration/test_parallel_replicas_working_set/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_parallel_replicas_working_set/configs/remote_servers.xml b/tests/integration/test_parallel_replicas_working_set/configs/remote_servers.xml deleted file mode 100644 index 02a315479f8..00000000000 --- a/tests/integration/test_parallel_replicas_working_set/configs/remote_servers.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - - - true - - n1 - 9000 - - - n2 - 9000 - - - n3 - 9000 - - - - - - diff --git a/tests/integration/test_parallel_replicas_working_set/test.py b/tests/integration/test_parallel_replicas_working_set/test.py deleted file mode 100644 index 0ede9d9b1a5..00000000000 --- a/tests/integration/test_parallel_replicas_working_set/test.py +++ /dev/null @@ -1,140 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -nodes = [ - cluster.add_instance( - f"n{i}", main_configs=["configs/remote_servers.xml"], with_zookeeper=True - ) - for i in (1, 2, 3) -] - - -@pytest.fixture(scope="module", autouse=True) -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def create_tables(cluster, table_name, node_with_covering_part): - # create replicated tables - for node in nodes: - node.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - - nodes[0].query( - f"""CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') - ORDER BY (key)""" - ) - nodes[1].query( - f"""CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') - ORDER BY (key)""" - ) - nodes[2].query( - f"""CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') - ORDER BY (key)""" - ) - # stop merges to keep original parts - # stop fetches to keep only parts created on the nodes - for i in (0, 1, 2): - if i != node_with_covering_part: - nodes[i].query(f"system stop fetches {table_name}") - nodes[i].query(f"system stop merges {table_name}") - - # populate data, equal number of rows for each replica - nodes[0].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(10)", - ) - nodes[0].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(10, 10)" - ) - nodes[1].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(20, 10)" - ) - nodes[1].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(30, 10)" - ) - nodes[2].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(40, 10)" - ) - nodes[2].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(50, 10)" - ) - nodes[node_with_covering_part].query(f"system sync replica {table_name}") - nodes[node_with_covering_part].query(f"optimize table {table_name}") - - # check we have expected set of parts - expected_active_parts = "" - if node_with_covering_part == 0: - expected_active_parts = ( - "all_0_5_1\nall_2_2_0\nall_3_3_0\nall_4_4_0\nall_5_5_0\n" - ) - - if node_with_covering_part == 1: - expected_active_parts = ( - "all_0_0_0\nall_0_5_1\nall_1_1_0\nall_4_4_0\nall_5_5_0\n" - ) - - if node_with_covering_part == 2: - expected_active_parts = ( - "all_0_0_0\nall_0_5_1\nall_1_1_0\nall_2_2_0\nall_3_3_0\n" - ) - - assert ( - nodes[0].query( - f"select distinct name from clusterAllReplicas({cluster}, system.parts) where table='{table_name}' and active order by name" - ) - == expected_active_parts - ) - - -@pytest.mark.parametrize("node_with_covering_part", [0, 1, 2]) -def test_covering_part_in_announcement(start_cluster, node_with_covering_part): - """create and populate table in special way (see create_table()), - node_with_covering_part contains all parts merged into one, - other nodes contain only parts which are result of insert via the node - """ - - cluster = "test_single_shard_multiple_replicas" - table_name = "test_table" - create_tables(cluster, table_name, node_with_covering_part) - - # query result can be one of the following outcomes - # (1) query result if parallel replicas working set contains all_0_5_1 - expected_full_result = "60\t0\t59\t1770\n" - expected_results = {expected_full_result} - - # (2) query result if parallel replicas working set DOESN'T contain all_0_5_1 - if node_with_covering_part == 0: - expected_results.add("40\t20\t59\t1580\n") - if node_with_covering_part == 1: - expected_results.add("40\t0\t59\t1180\n") - if node_with_covering_part == 2: - expected_results.add("40\t0\t39\t780\n") - - # parallel replicas - result = nodes[0].query( - f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}", - settings={ - "allow_experimental_parallel_reading_from_replicas": 2, - "prefer_localhost_replica": 0, - "max_parallel_replicas": 3, - "use_hedged_requests": 0, - "cluster_for_parallel_replicas": cluster, - }, - ) - assert result in expected_results - - # w/o parallel replicas - assert ( - nodes[node_with_covering_part].query( - f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}", - settings={ - "allow_experimental_parallel_reading_from_replicas": 0, - }, - ) - == expected_full_result - ) From 3c7ae2f171bb8bf56d04677448a6ab0384f865a5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 4 Jan 2024 11:20:07 +0000 Subject: [PATCH 46/61] Reduce bounding_ratio.xml --- tests/performance/bounding_ratio.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/bounding_ratio.xml b/tests/performance/bounding_ratio.xml index e3a15f90013..ed0b25848df 100644 --- a/tests/performance/bounding_ratio.xml +++ b/tests/performance/bounding_ratio.xml @@ -1,4 +1,4 @@ - SELECT boundingRatio(number, number) FROM numbers(100000000) - SELECT (argMax(number, number) - argMin(number, number)) / (max(number) - min(number)) FROM numbers(100000000) + SELECT boundingRatio(number, number) FROM numbers(30000000) + SELECT (argMax(number, number) - argMin(number, number)) / (max(number) - min(number)) FROM numbers(30000000) From 39eaa8dc9cd599b337a091dafa8cd3bb020e1b47 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 4 Jan 2024 11:24:36 +0000 Subject: [PATCH 47/61] Halve the size of reinterpret_as.xml --- tests/performance/reinterpret_as.xml | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/tests/performance/reinterpret_as.xml b/tests/performance/reinterpret_as.xml index dbf6df160ed..d05ef3bb038 100644 --- a/tests/performance/reinterpret_as.xml +++ b/tests/performance/reinterpret_as.xml @@ -19,7 +19,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -38,7 +38,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -57,7 +57,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -76,7 +76,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -95,7 +95,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(10000000) + FROM numbers_mt(5000000) SETTINGS max_threads = 8 FORMAT Null @@ -115,7 +115,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -134,7 +134,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -153,7 +153,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -172,7 +172,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -191,7 +191,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(100000000) + FROM numbers_mt(50000000) SETTINGS max_threads = 8 FORMAT Null @@ -210,7 +210,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(10000000) + FROM numbers_mt(5000000) SETTINGS max_threads = 8 FORMAT Null @@ -230,7 +230,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(20000000) + FROM numbers_mt(10000000) SETTINGS max_threads = 8 FORMAT Null @@ -249,7 +249,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(100000000) + FROM numbers_mt(50000000) SETTINGS max_threads = 8 FORMAT Null From 2aa6690f2c63c4630c04b6cae54e0fdbb8b12082 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 4 Jan 2024 11:29:17 +0000 Subject: [PATCH 48/61] Reduce hashed_dictionary.xml --- tests/performance/hashed_dictionary.xml | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/tests/performance/hashed_dictionary.xml b/tests/performance/hashed_dictionary.xml index e9038e694c6..b9de02a70e0 100644 --- a/tests/performance/hashed_dictionary.xml +++ b/tests/performance/hashed_dictionary.xml @@ -82,7 +82,6 @@ elements_count 5000000 - 7500000 @@ -90,16 +89,14 @@ WITH rand64() % toUInt64({elements_count}) as key SELECT dictGet('default.simple_key_hashed_dictionary', {column_name}, key) - FROM system.numbers - LIMIT {elements_count} + FROM numbers_mt({elements_count}) FORMAT Null; WITH rand64() % toUInt64({elements_count}) as key SELECT dictHas('default.simple_key_hashed_dictionary', key) - FROM system.numbers - LIMIT {elements_count} + FROM numbers_mt({elements_count}) FORMAT Null; @@ -111,16 +108,14 @@ WITH (rand64() % toUInt64({elements_count}), toString(rand64() % toUInt64({elements_count}))) as key SELECT dictGet('default.complex_key_hashed_dictionary', {column_name}, key) - FROM system.numbers - LIMIT {elements_count} + FROM numbers_mt({elements_count}) FORMAT Null; WITH (rand64() % toUInt64({elements_count}), toString(rand64() % toUInt64({elements_count}))) as key SELECT dictHas('default.complex_key_hashed_dictionary', key) - FROM system.numbers - LIMIT {elements_count} + FROM numbers_mt({elements_count}) FORMAT Null; From 1d1edd5b57b6f6cf188c6c616d09b374a9144268 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 4 Jan 2024 11:31:20 +0000 Subject: [PATCH 49/61] Reduce sum_map.xml --- tests/performance/sum_map.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/sum_map.xml b/tests/performance/sum_map.xml index f55af077023..ffb9b9507ae 100644 --- a/tests/performance/sum_map.xml +++ b/tests/performance/sum_map.xml @@ -7,7 +7,7 @@ scale - 1000000 + 100000 From 641caba5b0d1caf6a4146c769ee3af6b55bd8899 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 4 Jan 2024 11:36:33 +0000 Subject: [PATCH 50/61] Adapt more tests --- tests/performance/group_by_fixed_keys.xml | 2 +- tests/performance/join_used_flags.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/group_by_fixed_keys.xml b/tests/performance/group_by_fixed_keys.xml index a64208eb3de..d74b65ad47a 100644 --- a/tests/performance/group_by_fixed_keys.xml +++ b/tests/performance/group_by_fixed_keys.xml @@ -11,7 +11,7 @@ create table group_by_fk(a UInt32, b UInt32, c LowCardinality(UInt32), d Nullable(UInt32), e UInt64, f UInt64, g UInt64, h LowCardinality(UInt64), i Nullable(UInt64)) engine=MergeTree order by tuple() - insert into group_by_fk select number, number, number % 10000, number % 2 == 0 ? number : Null, number, number, number, number % 10000, number % 2 == 0 ? number : Null from numbers_mt(3e7) + insert into group_by_fk select number, number, number % 10000, number % 2 == 0 ? number : Null, number, number, number, number % 10000, number % 2 == 0 ? number : Null from numbers_mt(1e7) settings max_insert_threads=8 select a, b from group_by_fk group by a, b format Null diff --git a/tests/performance/join_used_flags.xml b/tests/performance/join_used_flags.xml index 70b0b45391d..1bb994f7be2 100644 --- a/tests/performance/join_used_flags.xml +++ b/tests/performance/join_used_flags.xml @@ -1,6 +1,6 @@ CREATE TABLE test_join_used_flags (i64 Int64, i32 Int32) ENGINE = Memory - INSERT INTO test_join_used_flags SELECT number AS i64, rand32() AS i32 FROM numbers_mt(3000000) + INSERT INTO test_join_used_flags SELECT number AS i64, rand32() AS i32 FROM numbers_mt(1500000) SELECT l.i64, r.i64, l.i32, r.i32 FROM test_join_used_flags l RIGHT JOIN test_join_used_flags r USING i64 format Null DROP TABLE IF EXISTS test_join_used_flags From b5997e6a9639f54698cf1dda354625a5f20bb776 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 4 Jan 2024 15:06:38 +0300 Subject: [PATCH 51/61] MergeTreePrefetchedReadPool disable for LIMIT only queries --- src/Interpreters/InterpreterSelectQuery.cpp | 7 ++++++- src/Planner/PlannerJoinTree.cpp | 7 ++++++- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 8 +++++++- 3 files changed, 19 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cdf1b4228bc..d3d7470ad25 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2501,7 +2501,12 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc max_block_size = std::max(1, max_block_limited); max_threads_execute_query = max_streams = 1; } - if (max_block_limited < local_limits.local_limits.size_limits.max_rows) + if (local_limits.local_limits.size_limits.max_rows != 0 && + max_block_limited < local_limits.local_limits.size_limits.max_rows) + { + query_info.limit = max_block_limited; + } + else { query_info.limit = max_block_limited; } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index e2cdf146a69..095db09ffbd 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -645,7 +645,12 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres max_threads_execute_query = 1; } - if (max_block_size_limited < select_query_info.local_storage_limits.local_limits.size_limits.max_rows) + if (select_query_info.local_storage_limits.local_limits.size_limits.max_rows != 0 && + max_block_size_limited < select_query_info.local_storage_limits.local_limits.size_limits.max_rows) + { + table_expression_query_info.limit = max_block_size_limited; + } + else { table_expression_query_info.limit = max_block_size_limited; } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index aa1c463e4e6..bdb2f7ea009 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -418,7 +418,13 @@ Pipe ReadFromMergeTree::readFromPool( && settings.allow_prefetched_read_pool_for_local_filesystem && MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.local_fs_method); - if (allow_prefetched_remote || allow_prefetched_local) + /** Do not use prefetched read pool if query is trivial limit query. + * Because time spend during filling per thread tasks can be greater than whole query + * execution for big tables with small limit. + */ + bool use_prefetched_read_pool = query_info.limit != 0 && (allow_prefetched_remote || allow_prefetched_local); + + if (use_prefetched_read_pool) { pool = std::make_shared( std::move(parts_with_range), From 8573c66b09d3879d65069d5b50713ad0714238b5 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 4 Jan 2024 15:29:25 +0300 Subject: [PATCH 52/61] Fixed code review issues --- src/Interpreters/InterpreterSelectQuery.cpp | 6 +++--- src/Planner/PlannerJoinTree.cpp | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d3d7470ad25..b6c9b8cdba3 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2501,10 +2501,10 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc max_block_size = std::max(1, max_block_limited); max_threads_execute_query = max_streams = 1; } - if (local_limits.local_limits.size_limits.max_rows != 0 && - max_block_limited < local_limits.local_limits.size_limits.max_rows) + if (local_limits.local_limits.size_limits.max_rows != 0) { - query_info.limit = max_block_limited; + if (max_block_limited < local_limits.local_limits.size_limits.max_rows) + query_info.limit = max_block_limited; } else { diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 095db09ffbd..857fb993600 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -645,10 +645,10 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres max_threads_execute_query = 1; } - if (select_query_info.local_storage_limits.local_limits.size_limits.max_rows != 0 && - max_block_size_limited < select_query_info.local_storage_limits.local_limits.size_limits.max_rows) + if (select_query_info.local_storage_limits.local_limits.size_limits.max_rows != 0) { - table_expression_query_info.limit = max_block_size_limited; + if (max_block_size_limited < select_query_info.local_storage_limits.local_limits.size_limits.max_rows) + table_expression_query_info.limit = max_block_size_limited; } else { From 74fb390444baec49360f5e07a34b32f63684218c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 4 Jan 2024 13:36:42 +0100 Subject: [PATCH 53/61] fix build --- src/Interpreters/DDLTask.cpp | 2 +- src/Interpreters/DDLTask.h | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 85bf6fec655..d418be51cc5 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -232,7 +232,7 @@ bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, c throw Exception( ErrorCodes::DNS_ERROR, "{} is not a local address. Check parameter 'host_name' in the configuration", - *config_host_name) + *config_host_name); } for (const HostID & host : entry.hosts) diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index e1a81ac97af..bc45b46bf0f 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -44,6 +44,9 @@ struct HostID explicit HostID(const Cluster::Address & address) : host_name(address.host_name), port(address.port) {} + HostID(const String & host_name_, UInt16 port_) + : host_name(host_name_), port(port_) {} + static HostID fromString(const String & host_port_str); String toString() const From 5bfddfebb6ac1f50ebbdca5d0e146f72fe085793 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 4 Jan 2024 14:08:58 +0000 Subject: [PATCH 54/61] Fix instantiation detection --- src/AggregateFunctions/AggregateFunctionMax.cpp | 5 +++-- src/AggregateFunctions/AggregateFunctionMin.cpp | 5 +++-- src/AggregateFunctions/AggregateFunctionMinMaxAny.h | 2 ++ 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMax.cpp b/src/AggregateFunctions/AggregateFunctionMax.cpp index 2577c932592..e9cd651b8db 100644 --- a/src/AggregateFunctions/AggregateFunctionMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionMax.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include namespace DB @@ -74,7 +75,7 @@ void AggregateFunctionsSingleValueMax::addBatchSinglePlace( Arena * arena, ssize_t if_argument_pos) const { - if constexpr (!std::is_same_v || !std::is_same_v) + if constexpr (!is_any_of) { /// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's /// faster than doing a permutation @@ -169,7 +170,7 @@ void AggregateFunctionsSingleValueMax::addBatchSinglePlaceNotNull( Arena * arena, ssize_t if_argument_pos) const { - if constexpr (!std::is_same_v || !std::is_same_v) + if constexpr (!is_any_of) { /// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's /// faster than doing a permutation diff --git a/src/AggregateFunctions/AggregateFunctionMin.cpp b/src/AggregateFunctions/AggregateFunctionMin.cpp index 701101e7207..d767bd5c563 100644 --- a/src/AggregateFunctions/AggregateFunctionMin.cpp +++ b/src/AggregateFunctions/AggregateFunctionMin.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include @@ -75,7 +76,7 @@ void AggregateFunctionsSingleValueMin::addBatchSinglePlace( Arena * arena, ssize_t if_argument_pos) const { - if constexpr (!std::is_same_v || !std::is_same_v) + if constexpr (!is_any_of) { /// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's /// faster than doing a permutation @@ -170,7 +171,7 @@ void AggregateFunctionsSingleValueMin::addBatchSinglePlaceNotNull( Arena * arena, ssize_t if_argument_pos) const { - if constexpr (!std::is_same_v || !std::is_same_v) + if constexpr (!is_any_of) { /// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's /// faster than doing a permutation diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index b69a0b100a3..dec70861543 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -965,6 +965,7 @@ template struct AggregateFunctionMinData : Data { using Self = AggregateFunctionMinData; + using Impl = Data; bool changeIfBetter(const IColumn & column, size_t row_num, Arena * arena) { return this->changeIfLess(column, row_num, arena); } bool changeIfBetter(const Self & to, Arena * arena) { return this->changeIfLess(to, arena); } @@ -993,6 +994,7 @@ template struct AggregateFunctionMaxData : Data { using Self = AggregateFunctionMaxData; + using Impl = Data; bool changeIfBetter(const IColumn & column, size_t row_num, Arena * arena) { return this->changeIfGreater(column, row_num, arena); } bool changeIfBetter(const Self & to, Arena * arena) { return this->changeIfGreater(to, arena); } From d9f68f4a2c4e3fcdce8776af5d9ee2cf7a551f15 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 4 Jan 2024 17:16:47 +0300 Subject: [PATCH 55/61] Fixed tests --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index bdb2f7ea009..6f0429459cd 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -422,7 +422,7 @@ Pipe ReadFromMergeTree::readFromPool( * Because time spend during filling per thread tasks can be greater than whole query * execution for big tables with small limit. */ - bool use_prefetched_read_pool = query_info.limit != 0 && (allow_prefetched_remote || allow_prefetched_local); + bool use_prefetched_read_pool = query_info.limit == 0 && (allow_prefetched_remote || allow_prefetched_local); if (use_prefetched_read_pool) { From 494a32f4e47af2576455cda2794ffa13568c60f3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 4 Jan 2024 14:41:04 +0000 Subject: [PATCH 56/61] Review fixes --- src/Storages/StorageS3.cpp | 105 ++++++++----------------------------- 1 file changed, 23 insertions(+), 82 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ce49be32120..d7cc86ed321 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -147,7 +147,8 @@ public: const Names & column_names_, StorageSnapshotPtr storage_snapshot_, StorageS3 & storage_, - SelectQueryInfo query_info_, + ReadFromFormatInfo read_from_format_info_, + bool need_only_count_, ContextPtr context_, size_t max_block_size_, size_t num_streams_) @@ -155,7 +156,8 @@ public: , column_names(column_names_) , storage_snapshot(std::move(storage_snapshot_)) , storage(storage_) - , query_info(std::move(query_info_)) + , read_from_format_info(std::move(read_from_format_info_)) + , need_only_count(need_only_count_) , local_context(std::move(context_)) , max_block_size(max_block_size_) , num_streams(num_streams_) @@ -168,7 +170,8 @@ private: Names column_names; StorageSnapshotPtr storage_snapshot; StorageS3 & storage; - SelectQueryInfo query_info; + ReadFromFormatInfo read_from_format_info; + bool need_only_count; StorageS3::Configuration query_configuration; NamesAndTypesList virtual_columns; @@ -183,77 +186,6 @@ private: }; -static Block getBlockWithVirtuals(const NamesAndTypesList & virtual_columns, const String & bucket, const std::unordered_set & keys) -{ - Block virtual_columns_block; - fs::path bucket_path(bucket); - - for (const auto & [column_name, column_type] : virtual_columns) - { - if (column_name == "_path") - { - auto column = column_type->createColumn(); - for (const auto & key : keys) - column->insert((bucket_path / key).string()); - virtual_columns_block.insert({std::move(column), column_type, column_name}); - } - else if (column_name == "_file") - { - auto column = column_type->createColumn(); - for (const auto & key : keys) - { - auto pos = key.find_last_of('/'); - if (pos != std::string::npos) - column->insert(key.substr(pos + 1)); - else - column->insert(key); - } - virtual_columns_block.insert({std::move(column), column_type, column_name}); - } - else if (column_name == "_key") - { - auto column = column_type->createColumn(); - for (const auto & key : keys) - column->insert(key); - virtual_columns_block.insert({std::move(column), column_type, column_name}); - } - else - { - auto column = column_type->createColumn(); - column->insertManyDefaults(keys.size()); - virtual_columns_block.insert({std::move(column), column_type, column_name}); - } - } - - /// Column _key is mandatory and may not be in virtual_columns list - if (!virtual_columns_block.has("_key")) - { - auto column_type = std::make_shared(); - auto column = column_type->createColumn(); for (const auto & key : keys) - column->insert(key); - virtual_columns_block.insert({std::move(column), column_type, "_key"}); - } - - return virtual_columns_block; -} - -static std::vector filterKeysForPartitionPruning( - const std::vector & keys, - const String & bucket, - const NamesAndTypesList & virtual_columns, - const ActionsDAG::Node * predicate, - ContextPtr context) -{ - std::unordered_set result_keys(keys.begin(), keys.end()); - - auto block = getBlockWithVirtuals(virtual_columns, bucket, result_keys); - VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); - result_keys = VirtualColumnUtils::extractSingleValueFromBlock(block, "_key"); - - LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Applied partition pruning {} from {} keys left", result_keys.size(), keys.size()); - return std::vector(result_keys.begin(), result_keys.end()); -} - class IOutputFormat; using OutputFormatPtr = std::shared_ptr; @@ -305,9 +237,9 @@ public: "Cannot compile regex from glob ({}): {}", globbed_uri.key, matcher->error()); recursive = globbed_uri.key == "/**" ? true : false; - fillInternalBufferAssumeLocked(); filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + fillInternalBufferAssumeLocked(); } KeyWithInfoPtr next() @@ -1161,7 +1093,17 @@ static std::shared_ptr createFileIterator( } else { - Strings keys = filterKeysForPartitionPruning(configuration.keys, configuration.url.bucket, virtual_columns, predicate, local_context); + Strings keys = configuration.keys; + auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + if (filter_dag) + { + std::vector paths; + paths.reserve(keys.size()); + for (const auto & key : keys) + paths.push_back(fs::path(configuration.url.bucket) / key); + VirtualColumnUtils::filterByPathOrFile(keys, paths, filter_dag, virtual_columns, local_context); + } + return std::make_shared( *configuration.client, configuration.url.version_id, keys, configuration.url.bucket, configuration.request_settings, read_keys, file_progress_callback); @@ -1195,12 +1137,16 @@ void StorageS3::read( { auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), virtual_columns); + bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) + && local_context->getSettingsRef().optimize_count_from_files; + auto reading = std::make_unique( read_from_format_info.source_header, column_names, storage_snapshot, *this, - query_info, + std::move(read_from_format_info), + need_only_count, local_context, max_block_size, num_streams); @@ -1235,8 +1181,6 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, createIterator(nullptr); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, storage.supportsSubsetOfColumns(local_context), virtual_columns); - size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); if (estimated_keys_count > 1) num_streams = std::min(num_streams, estimated_keys_count); @@ -1244,9 +1188,6 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, /// Disclosed glob iterator can underestimate the amount of keys in some cases. We will keep one stream for this particular case. num_streams = 1; - bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) - && local_context->getSettingsRef().optimize_count_from_files; - const size_t max_threads = local_context->getSettingsRef().max_threads; const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul)); LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Reading in {} streams, {} threads per stream", num_streams, max_parsing_threads); From 296e1ac8aa000996f004343aa299e5b732c7c8df Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Jan 2024 16:11:39 +0000 Subject: [PATCH 57/61] FunctionSqid.cpp --> sqid.cpp --- src/Functions/{FunctionSqid.cpp => sqid.cpp} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename src/Functions/{FunctionSqid.cpp => sqid.cpp} (100%) diff --git a/src/Functions/FunctionSqid.cpp b/src/Functions/sqid.cpp similarity index 100% rename from src/Functions/FunctionSqid.cpp rename to src/Functions/sqid.cpp From 03e344c36ae27b62cfcf058640b81b8ae8460afe Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Jan 2024 16:15:06 +0000 Subject: [PATCH 58/61] Fix preprocessor guard --- src/Functions/sqid.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/sqid.cpp b/src/Functions/sqid.cpp index 546263914c2..4517bba963e 100644 --- a/src/Functions/sqid.cpp +++ b/src/Functions/sqid.cpp @@ -1,6 +1,6 @@ #include "config.h" -#ifdef ENABLE_SQIDS +#if USE_SQIDS #include #include From 98d602c3d5e3e197ed9d3579ad34155b386acb74 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Jan 2024 16:21:08 +0000 Subject: [PATCH 59/61] Reserve enough space in result column upfront --- src/Functions/sqid.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Functions/sqid.cpp b/src/Functions/sqid.cpp index 4517bba963e..abd9d22f4c5 100644 --- a/src/Functions/sqid.cpp +++ b/src/Functions/sqid.cpp @@ -57,9 +57,10 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - size_t num_args = arguments.size(); auto col_res = ColumnString::create(); + col_res->reserve(input_rows_count); + const size_t num_args = arguments.size(); std::vector numbers(num_args); for (size_t i = 0; i < input_rows_count; ++i) { From 52058211e7ff227feb9c890f641d2299af9a246c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 4 Jan 2024 08:21:46 -0800 Subject: [PATCH 60/61] Fix some thread pool settings not updating at runtime (#58485) --- programs/server/Server.cpp | 94 +++++++++++++++++++------------------- 1 file changed, 47 insertions(+), 47 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 926e57070f3..1fa3d1cfa73 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1260,11 +1260,11 @@ try { Settings::checkNoSettingNamesAtTopLevel(*config, config_path); - ServerSettings server_settings_; - server_settings_.loadSettingsFromConfig(*config); + ServerSettings new_server_settings; + new_server_settings.loadSettingsFromConfig(*config); - size_t max_server_memory_usage = server_settings_.max_server_memory_usage; - double max_server_memory_usage_to_ram_ratio = server_settings_.max_server_memory_usage_to_ram_ratio; + size_t max_server_memory_usage = new_server_settings.max_server_memory_usage; + double max_server_memory_usage_to_ram_ratio = new_server_settings.max_server_memory_usage_to_ram_ratio; size_t current_physical_server_memory = getMemoryAmount(); /// With cgroups, the amount of memory available to the server can be changed dynamically. size_t default_max_server_memory_usage = static_cast(current_physical_server_memory * max_server_memory_usage_to_ram_ratio); @@ -1294,9 +1294,9 @@ try total_memory_tracker.setDescription("(total)"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); - size_t merges_mutations_memory_usage_soft_limit = server_settings_.merges_mutations_memory_usage_soft_limit; + size_t merges_mutations_memory_usage_soft_limit = new_server_settings.merges_mutations_memory_usage_soft_limit; - size_t default_merges_mutations_server_memory_usage = static_cast(current_physical_server_memory * server_settings_.merges_mutations_memory_usage_to_ram_ratio); + size_t default_merges_mutations_server_memory_usage = static_cast(current_physical_server_memory * new_server_settings.merges_mutations_memory_usage_to_ram_ratio); if (merges_mutations_memory_usage_soft_limit == 0) { merges_mutations_memory_usage_soft_limit = default_merges_mutations_server_memory_usage; @@ -1304,7 +1304,7 @@ try " ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio)", formatReadableSizeWithBinarySuffix(merges_mutations_memory_usage_soft_limit), formatReadableSizeWithBinarySuffix(current_physical_server_memory), - server_settings_.merges_mutations_memory_usage_to_ram_ratio); + new_server_settings.merges_mutations_memory_usage_to_ram_ratio); } else if (merges_mutations_memory_usage_soft_limit > default_merges_mutations_server_memory_usage) { @@ -1313,7 +1313,7 @@ try " ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio)", formatReadableSizeWithBinarySuffix(merges_mutations_memory_usage_soft_limit), formatReadableSizeWithBinarySuffix(current_physical_server_memory), - server_settings_.merges_mutations_memory_usage_to_ram_ratio); + new_server_settings.merges_mutations_memory_usage_to_ram_ratio); } LOG_INFO(log, "Merges and mutations memory limit is set to {}", @@ -1322,7 +1322,7 @@ try background_memory_tracker.setDescription("(background)"); background_memory_tracker.setMetric(CurrentMetrics::MergesMutationsMemoryTracking); - total_memory_tracker.setAllowUseJemallocMemory(server_settings_.allow_use_jemalloc_memory); + total_memory_tracker.setAllowUseJemallocMemory(new_server_settings.allow_use_jemalloc_memory); auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker(); total_memory_tracker.setOvercommitTracker(global_overcommit_tracker); @@ -1346,26 +1346,26 @@ try global_context->setRemoteHostFilter(*config); global_context->setHTTPHeaderFilter(*config); - global_context->setMaxTableSizeToDrop(server_settings_.max_table_size_to_drop); - global_context->setMaxPartitionSizeToDrop(server_settings_.max_partition_size_to_drop); - global_context->setMaxTableNumToWarn(server_settings_.max_table_num_to_warn); - global_context->setMaxDatabaseNumToWarn(server_settings_.max_database_num_to_warn); - global_context->setMaxPartNumToWarn(server_settings_.max_part_num_to_warn); + global_context->setMaxTableSizeToDrop(new_server_settings.max_table_size_to_drop); + global_context->setMaxPartitionSizeToDrop(new_server_settings.max_partition_size_to_drop); + global_context->setMaxTableNumToWarn(new_server_settings.max_table_num_to_warn); + global_context->setMaxDatabaseNumToWarn(new_server_settings.max_database_num_to_warn); + global_context->setMaxPartNumToWarn(new_server_settings.max_part_num_to_warn); ConcurrencyControl::SlotCount concurrent_threads_soft_limit = ConcurrencyControl::Unlimited; - if (server_settings_.concurrent_threads_soft_limit_num > 0 && server_settings_.concurrent_threads_soft_limit_num < concurrent_threads_soft_limit) - concurrent_threads_soft_limit = server_settings_.concurrent_threads_soft_limit_num; - if (server_settings_.concurrent_threads_soft_limit_ratio_to_cores > 0) + if (new_server_settings.concurrent_threads_soft_limit_num > 0 && new_server_settings.concurrent_threads_soft_limit_num < concurrent_threads_soft_limit) + concurrent_threads_soft_limit = new_server_settings.concurrent_threads_soft_limit_num; + if (new_server_settings.concurrent_threads_soft_limit_ratio_to_cores > 0) { - auto value = server_settings_.concurrent_threads_soft_limit_ratio_to_cores * std::thread::hardware_concurrency(); + auto value = new_server_settings.concurrent_threads_soft_limit_ratio_to_cores * std::thread::hardware_concurrency(); if (value > 0 && value < concurrent_threads_soft_limit) concurrent_threads_soft_limit = value; } ConcurrencyControl::instance().setMaxConcurrency(concurrent_threads_soft_limit); - global_context->getProcessList().setMaxSize(server_settings_.max_concurrent_queries); - global_context->getProcessList().setMaxInsertQueriesAmount(server_settings_.max_concurrent_insert_queries); - global_context->getProcessList().setMaxSelectQueriesAmount(server_settings_.max_concurrent_select_queries); + global_context->getProcessList().setMaxSize(new_server_settings.max_concurrent_queries); + global_context->getProcessList().setMaxInsertQueriesAmount(new_server_settings.max_concurrent_insert_queries); + global_context->getProcessList().setMaxSelectQueriesAmount(new_server_settings.max_concurrent_select_queries); if (config->has("keeper_server")) global_context->updateKeeperConfiguration(*config); @@ -1376,68 +1376,68 @@ try /// This is done for backward compatibility. if (global_context->areBackgroundExecutorsInitialized()) { - auto new_pool_size = server_settings_.background_pool_size; - auto new_ratio = server_settings_.background_merges_mutations_concurrency_ratio; + auto new_pool_size = new_server_settings.background_pool_size; + auto new_ratio = new_server_settings.background_merges_mutations_concurrency_ratio; global_context->getMergeMutateExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, static_cast(new_pool_size * new_ratio)); - global_context->getMergeMutateExecutor()->updateSchedulingPolicy(server_settings_.background_merges_mutations_scheduling_policy.toString()); + global_context->getMergeMutateExecutor()->updateSchedulingPolicy(new_server_settings.background_merges_mutations_scheduling_policy.toString()); } if (global_context->areBackgroundExecutorsInitialized()) { - auto new_pool_size = server_settings_.background_move_pool_size; + auto new_pool_size = new_server_settings.background_move_pool_size; global_context->getMovesExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size); } if (global_context->areBackgroundExecutorsInitialized()) { - auto new_pool_size = server_settings_.background_fetches_pool_size; + auto new_pool_size = new_server_settings.background_fetches_pool_size; global_context->getFetchesExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size); } if (global_context->areBackgroundExecutorsInitialized()) { - auto new_pool_size = server_settings_.background_common_pool_size; + auto new_pool_size = new_server_settings.background_common_pool_size; global_context->getCommonExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size); } - global_context->getBufferFlushSchedulePool().increaseThreadsCount(server_settings_.background_buffer_flush_schedule_pool_size); - global_context->getSchedulePool().increaseThreadsCount(server_settings_.background_schedule_pool_size); - global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings_.background_message_broker_schedule_pool_size); - global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings_.background_distributed_schedule_pool_size); + global_context->getBufferFlushSchedulePool().increaseThreadsCount(new_server_settings.background_buffer_flush_schedule_pool_size); + global_context->getSchedulePool().increaseThreadsCount(new_server_settings.background_schedule_pool_size); + global_context->getMessageBrokerSchedulePool().increaseThreadsCount(new_server_settings.background_message_broker_schedule_pool_size); + global_context->getDistributedSchedulePool().increaseThreadsCount(new_server_settings.background_distributed_schedule_pool_size); - global_context->getAsyncLoader().setMaxThreads(TablesLoaderForegroundPoolId, server_settings_.tables_loader_foreground_pool_size); - global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundLoadPoolId, server_settings_.tables_loader_background_pool_size); - global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundStartupPoolId, server_settings_.tables_loader_background_pool_size); + global_context->getAsyncLoader().setMaxThreads(TablesLoaderForegroundPoolId, new_server_settings.tables_loader_foreground_pool_size); + global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundLoadPoolId, new_server_settings.tables_loader_background_pool_size); + global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundStartupPoolId, new_server_settings.tables_loader_background_pool_size); getIOThreadPool().reloadConfiguration( - server_settings.max_io_thread_pool_size, - server_settings.max_io_thread_pool_free_size, - server_settings.io_thread_pool_queue_size); + new_server_settings.max_io_thread_pool_size, + new_server_settings.max_io_thread_pool_free_size, + new_server_settings.io_thread_pool_queue_size); getBackupsIOThreadPool().reloadConfiguration( - server_settings.max_backups_io_thread_pool_size, - server_settings.max_backups_io_thread_pool_free_size, - server_settings.backups_io_thread_pool_queue_size); + new_server_settings.max_backups_io_thread_pool_size, + new_server_settings.max_backups_io_thread_pool_free_size, + new_server_settings.backups_io_thread_pool_queue_size); getActivePartsLoadingThreadPool().reloadConfiguration( - server_settings.max_active_parts_loading_thread_pool_size, + new_server_settings.max_active_parts_loading_thread_pool_size, 0, // We don't need any threads once all the parts will be loaded - server_settings.max_active_parts_loading_thread_pool_size); + new_server_settings.max_active_parts_loading_thread_pool_size); getOutdatedPartsLoadingThreadPool().reloadConfiguration( - server_settings.max_outdated_parts_loading_thread_pool_size, + new_server_settings.max_outdated_parts_loading_thread_pool_size, 0, // We don't need any threads once all the parts will be loaded - server_settings.max_outdated_parts_loading_thread_pool_size); + new_server_settings.max_outdated_parts_loading_thread_pool_size); /// It could grow if we need to synchronously wait until all the data parts will be loaded. getOutdatedPartsLoadingThreadPool().setMaxTurboThreads( - server_settings.max_active_parts_loading_thread_pool_size + new_server_settings.max_active_parts_loading_thread_pool_size ); getPartsCleaningThreadPool().reloadConfiguration( - server_settings.max_parts_cleaning_thread_pool_size, + new_server_settings.max_parts_cleaning_thread_pool_size, 0, // We don't need any threads one all the parts will be deleted - server_settings.max_parts_cleaning_thread_pool_size); + new_server_settings.max_parts_cleaning_thread_pool_size); if (config->has("resources")) { From 76b7cddb186ba6d44e581fa35dce9fd48fc6b3ed Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Jan 2024 16:29:43 +0000 Subject: [PATCH 61/61] Update docs --- docs/en/sql-reference/functions/hash-functions.md | 4 +++- src/Functions/sqid.cpp | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index a23849c13aa..2c6a468af0e 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -1779,7 +1779,9 @@ Result: ## sqid -Transforms numbers into YouTube-like short URL hash called [Sqid](https://sqids.org/). +Transforms numbers into a [Sqid](https://sqids.org/) which is a YouTube-like ID string. +The output alphabet is `abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789`. +Do not use this function for hashing - the generated IDs can be decoded back into numbers. **Syntax** diff --git a/src/Functions/sqid.cpp b/src/Functions/sqid.cpp index abd9d22f4c5..363a3f8ac13 100644 --- a/src/Functions/sqid.cpp +++ b/src/Functions/sqid.cpp @@ -84,7 +84,7 @@ REGISTER_FUNCTION(Sqid) { factory.registerFunction(FunctionDocumentation{ .description=R"( -Transforms numbers into YouTube-like short URL hash called [Sqid](https://sqids.org/).)", +Transforms numbers into a [Sqid](https://sqids.org/) which is a Youtube-like ID string.)", .syntax="sqid(number1, ...)", .arguments={{"number1, ...", "Arbitrarily many UInt8, UInt16, UInt32 or UInt64 arguments"}}, .returned_value="A hash id [String](/docs/en/sql-reference/data-types/string.md).",