diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index af578423b043..b08d396d1d56 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -240,7 +240,7 @@ function run_tests() ) if [[ -n "$USE_PARALLEL_REPLICAS" ]] && [[ "$USE_PARALLEL_REPLICAS" -eq 1 ]]; then TEST_ARGS+=( - --client="clickhouse-client --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 --max_parallel_replicas=100 --cluster_for_parallel_replicas='parallel_replicas'" + --client="clickhouse-client --enable_parallel_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 --max_parallel_replicas=100 --cluster_for_parallel_replicas='parallel_replicas'" --no-parallel-replicas ) fi diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 5c2af401b049..56c983c5a395 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1588,25 +1588,12 @@ This setting is useful for any replicated table. An arbitrary integer expression that can be used to split work between replicas for a specific table. The value can be any integer expression. -A query may be processed faster if it is executed on several servers in parallel but it depends on the used [parallel_replicas_custom_key](#parallel_replicas_custom_key) -and [parallel_replicas_custom_key_filter_type](#parallel_replicas_custom_key_filter_type). Simple expressions using primary keys are preferred. If the setting is used on a cluster that consists of a single shard with multiple replicas, those replicas will be converted into virtual shards. Otherwise, it will behave same as for `SAMPLE` key, it will use multiple replicas of each shard. -## parallel_replicas_custom_key_filter_type {#parallel_replicas_custom_key_filter_type} - -How to use `parallel_replicas_custom_key` expression for splitting work between replicas. - -Possible values: - -- `default` — Use the default implementation using modulo operation on the `parallel_replicas_custom_key`. -- `range` — Split the entire value space of the expression in the ranges. This type of filtering is useful if values of `parallel_replicas_custom_key` are uniformly spread across the entire integer space, e.g. hash values. - -Default value: `default`. - ## parallel_replicas_custom_key_range_lower {#parallel_replicas_custom_key_range_lower} Allows the filter type `range` to split the work evenly between replicas based on the custom range `[parallel_replicas_custom_key_range_lower, INT_MAX]`. @@ -1621,9 +1608,9 @@ Allows the filter type `range` to split the work evenly between replicas based o When used in conjuction with [parallel_replicas_custom_key_range_lower](#parallel_replicas_custom_key_range_lower), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. -Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing. +Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing -## allow_experimental_parallel_reading_from_replicas +## enable_parallel_replicas Enables or disables sending SELECT queries to all replicas of a table (up to `max_parallel_replicas`). Reading is parallelized and coordinated dynamically. It will work for any kind of MergeTree table. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 84bbf6c83d31..a6b45f4a7920 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4215,3 +4215,9 @@ SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_pars │ 1.7091 │ 15008753 │ └─────────────────────┴──────────────────────────┘ ``` + +## push_external_roles_in_interserver_queries + +Позволяет передавать роли пользователя от инициатора запроса другим нодам при выполнении запроса. + +Значение по умолчанию: `true`. diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 48dca82eb2b1..36f774a3c12b 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -75,6 +75,8 @@ class Benchmark : public Poco::Util::Application const String & default_database_, const String & user_, const String & password_, + const String & proto_send_chunked_, + const String & proto_recv_chunked_, const String & quota_key_, const String & stage, bool randomize_, @@ -128,7 +130,9 @@ class Benchmark : public Poco::Util::Application connections.emplace_back(std::make_unique( concurrency, cur_host, cur_port, - default_database_, user_, password_, quota_key_, + default_database_, user_, password_, + proto_send_chunked_, proto_recv_chunked_, + quota_key_, /* cluster_= */ "", /* cluster_secret_= */ "", /* client_name_= */ std::string(DEFAULT_CLIENT_NAME), @@ -662,6 +666,50 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) Strings hosts = options.count("host") ? options["host"].as() : Strings({"localhost"}); + String proto_send_chunked {"notchunked"}; + String proto_recv_chunked {"notchunked"}; + + if (options.count("proto_caps")) + { + std::string proto_caps_str = options["proto_caps"].as(); + + std::vector proto_caps; + splitInto<','>(proto_caps, proto_caps_str); + + for (auto cap_str : proto_caps) + { + std::string direction; + + if (cap_str.starts_with("send_")) + { + direction = "send"; + cap_str = cap_str.substr(std::string_view("send_").size()); + } + else if (cap_str.starts_with("recv_")) + { + direction = "recv"; + cap_str = cap_str.substr(std::string_view("recv_").size()); + } + + if (cap_str != "chunked" && cap_str != "notchunked" && cap_str != "chunked_optional" && cap_str != "notchunked_optional") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "proto_caps option is incorrect ({})", proto_caps_str); + + if (direction.empty()) + { + proto_send_chunked = cap_str; + proto_recv_chunked = cap_str; + } + else + { + if (direction == "send") + proto_send_chunked = cap_str; + else + proto_recv_chunked = cap_str; + } + } + } + + Benchmark benchmark( options["concurrency"].as(), options["delay"].as(), @@ -673,6 +721,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) options["database"].as(), options["user"].as(), options["password"].as(), + proto_send_chunked, + proto_recv_chunked, options["quota_key"].as(), options["stage"].as(), options.count("randomize"), diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 25c94c56aa6d..d24f44bceb20 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -210,7 +210,7 @@ std::vector Client::loadWarningMessages() "" /* query_id */, QueryProcessingStage::Complete, &client_context->getSettingsRef(), - &client_context->getClientInfo(), false, {}); + &client_context->getClientInfo(), false, {}, {}); while (true) { Packet packet = connection->receivePacket(); diff --git a/programs/client/clickhouse-client.xml b/programs/client/clickhouse-client.xml index c32b63413e93..9ce7d1cb2233 100644 --- a/programs/client/clickhouse-client.xml +++ b/programs/client/clickhouse-client.xml @@ -38,6 +38,21 @@ {display_name} \e[1;31m:)\e[0m + + + 9000 + + + diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index e50521a07307..8ee777cf503f 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -366,6 +366,13 @@ void ContextAccess::setUser(const UserPtr & user_) const current_roles_with_admin_option = user->granted_roles.findGrantedWithAdminOption(*params.current_roles); } + if (params.external_roles && !params.external_roles->empty()) + { + current_roles.insert(current_roles.end(), params.external_roles->begin(), params.external_roles->end()); + auto new_granted_with_admin_option = user->granted_roles.findGrantedWithAdminOption(*params.external_roles); + current_roles_with_admin_option.insert(current_roles_with_admin_option.end(), new_granted_with_admin_option.begin(), new_granted_with_admin_option.end()); + } + subscription_for_roles_changes.reset(); enabled_roles = access_control->getEnabledRoles(current_roles, current_roles_with_admin_option); subscription_for_roles_changes = enabled_roles->subscribeForChanges([weak_ptr = weak_from_this()](const std::shared_ptr & roles_info_) @@ -516,7 +523,6 @@ std::optional ContextAccess::getQuotaUsage() const return getQuota()->getUsage(); } - SettingsChanges ContextAccess::getDefaultSettings() const { std::lock_guard lock{mutex}; diff --git a/src/Access/ContextAccessParams.cpp b/src/Access/ContextAccessParams.cpp index 07374e9bbf52..52d92c9b241f 100644 --- a/src/Access/ContextAccessParams.cpp +++ b/src/Access/ContextAccessParams.cpp @@ -11,6 +11,7 @@ ContextAccessParams::ContextAccessParams( bool full_access_, bool use_default_roles_, const std::shared_ptr> & current_roles_, + const std::shared_ptr> & external_roles_, const Settings & settings_, const String & current_database_, const ClientInfo & client_info_) @@ -18,6 +19,7 @@ ContextAccessParams::ContextAccessParams( , full_access(full_access_) , use_default_roles(use_default_roles_) , current_roles(current_roles_) + , external_roles(external_roles_) , readonly(settings_.readonly) , allow_ddl(settings_.allow_ddl) , allow_introspection(settings_.allow_introspection_functions) @@ -52,6 +54,17 @@ String ContextAccessParams::toString() const } out << "]"; } + if (external_roles && !external_roles->empty()) + { + out << separator() << "external_roles = ["; + for (size_t i = 0; i != external_roles->size(); ++i) + { + if (i) + out << ", "; + out << (*external_roles)[i]; + } + out << "]"; + } if (readonly) out << separator() << "readonly = " << readonly; if (allow_ddl) @@ -101,6 +114,7 @@ bool operator ==(const ContextAccessParams & left, const ContextAccessParams & r CONTEXT_ACCESS_PARAMS_EQUALS(full_access) CONTEXT_ACCESS_PARAMS_EQUALS(use_default_roles) CONTEXT_ACCESS_PARAMS_EQUALS(current_roles) + CONTEXT_ACCESS_PARAMS_EQUALS(external_roles) CONTEXT_ACCESS_PARAMS_EQUALS(readonly) CONTEXT_ACCESS_PARAMS_EQUALS(allow_ddl) CONTEXT_ACCESS_PARAMS_EQUALS(allow_introspection) @@ -153,6 +167,7 @@ bool operator <(const ContextAccessParams & left, const ContextAccessParams & ri CONTEXT_ACCESS_PARAMS_LESS(full_access) CONTEXT_ACCESS_PARAMS_LESS(use_default_roles) CONTEXT_ACCESS_PARAMS_LESS(current_roles) + CONTEXT_ACCESS_PARAMS_LESS(external_roles) CONTEXT_ACCESS_PARAMS_LESS(readonly) CONTEXT_ACCESS_PARAMS_LESS(allow_ddl) CONTEXT_ACCESS_PARAMS_LESS(allow_introspection) diff --git a/src/Access/ContextAccessParams.h b/src/Access/ContextAccessParams.h index 07503a3af6de..82592d630dd2 100644 --- a/src/Access/ContextAccessParams.h +++ b/src/Access/ContextAccessParams.h @@ -19,6 +19,7 @@ class ContextAccessParams bool full_access_, bool use_default_roles_, const std::shared_ptr> & current_roles_, + const std::shared_ptr> & external_roles_, const Settings & settings_, const String & current_database_, const ClientInfo & client_info_); @@ -31,6 +32,7 @@ class ContextAccessParams const bool use_default_roles; const std::shared_ptr> current_roles; + const std::shared_ptr> external_roles; const UInt64 readonly; const bool allow_ddl; diff --git a/src/Access/LDAPAccessStorage.cpp b/src/Access/LDAPAccessStorage.cpp index 3206b20b691b..917ad7cbb922 100644 --- a/src/Access/LDAPAccessStorage.cpp +++ b/src/Access/LDAPAccessStorage.cpp @@ -26,7 +26,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } - LDAPAccessStorage::LDAPAccessStorage(const String & storage_name_, AccessControl & access_control_, const Poco::Util::AbstractConfiguration & config, const String & prefix) : IAccessStorage(storage_name_), access_control(access_control_), memory_storage(storage_name_, access_control.getChangesNotifier(), false) { @@ -191,8 +190,8 @@ void LDAPAccessStorage::applyRoleChangeNoLock(bool grant, const UUID & role_id, } else { - granted_role_names.erase(role_id); granted_role_ids.erase(role_name); + granted_role_names.erase(role_id); } } @@ -320,6 +319,10 @@ std::set LDAPAccessStorage::mapExternalRolesNoLock(const LDAPClient::Sea { std::set role_names; + // If this node can't access LDAP server (or has not privileges to fetch roles) and gets empty list of external roles + if (external_roles.empty()) + return role_names; + if (external_roles.size() != role_search_params.size()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unable to map external roles"); diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index 71d13ad4f53a..0649aa5f5d7c 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -158,6 +158,8 @@ void ClientApplicationBase::init(int argc, char ** argv) ("config-file,C", po::value(), "config-file path") + ("proto_caps", po::value(), "enable/disable chunked protocol: chunked_optional, notchunked, notchunked_optional, send_chunked, send_chunked_optional, send_notchunked, send_notchunked_optional, recv_chunked, recv_chunked_optional, recv_notchunked, recv_notchunked_optional") + ("query,q", po::value>()->multitoken(), R"(Query. Can be specified multiple times (--query "SELECT 1" --query "SELECT 2") or once with multiple comma-separated queries (--query "SELECT 1; SELECT 2;"). In the latter case, INSERT queries with non-VALUE format must be separated by empty lines.)") ("queries-file", po::value>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)") ("multiquery,n", "Obsolete, does nothing") @@ -337,6 +339,41 @@ void ClientApplicationBase::init(int argc, char ** argv) if (options.count("server_logs_file")) server_logs_file = options["server_logs_file"].as(); + if (options.count("proto_caps")) + { + std::string proto_caps_str = options["proto_caps"].as(); + + std::vector proto_caps; + splitInto<','>(proto_caps, proto_caps_str); + + for (auto cap_str : proto_caps) + { + std::string direction; + + if (cap_str.starts_with("send_")) + { + direction = "send"; + cap_str = cap_str.substr(std::string_view("send_").size()); + } + else if (cap_str.starts_with("recv_")) + { + direction = "recv"; + cap_str = cap_str.substr(std::string_view("recv_").size()); + } + + if (cap_str != "chunked" && cap_str != "notchunked" && cap_str != "chunked_optional" && cap_str != "notchunked_optional") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "proto_caps option is incorrect ({})", proto_caps_str); + + if (direction.empty()) + { + config().setString("proto_caps.send", std::string(cap_str)); + config().setString("proto_caps.recv", std::string(cap_str)); + } + else + config().setString("proto_caps." + direction, std::string(cap_str)); + } + } + query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); query_kind = parseQueryKind(options["query_kind"].as()); profile_events.print = options.count("print-profile-events"); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 8f69eaf54b44..eb574b98edd5 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -73,9 +73,11 @@ #include #include #include +#include #include #include +#include #include "config.h" #include #include @@ -914,6 +916,8 @@ void ClientBase::processTextAsSingleQuery(const String & full_query) } catch (Exception & e) { + if (server_exception) + server_exception->rethrow(); if (!is_interactive) e.addMessage("(in query: {})", full_query); throw; @@ -1032,19 +1036,29 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa query_interrupt_handler.start(signals_before_stop); SCOPE_EXIT({ query_interrupt_handler.stop(); }); - connection->sendQuery( - connection_parameters.timeouts, - query, - query_parameters, - client_context->getCurrentQueryId(), - query_processing_stage, - &client_context->getSettingsRef(), - &client_context->getClientInfo(), - true, - [&](const Progress & progress) { onProgress(progress); }); - - if (send_external_tables) - sendExternalTables(parsed_query); + try { + connection->sendQuery( + connection_parameters.timeouts, + query, + query_parameters, + client_context->getCurrentQueryId(), + query_processing_stage, + &client_context->getSettingsRef(), + &client_context->getClientInfo(), + true, + {}, + [&](const Progress & progress) { onProgress(progress); }); + + if (send_external_tables) + sendExternalTables(parsed_query); + } + catch (const NetException &) + { + // We still want to attempt to process whatever we already received or can receive (socket receive buffer can be not empty) + receiveResult(parsed_query, signals_before_stop, settings.partial_result_on_first_cancel); + throw; + } + receiveResult(parsed_query, signals_before_stop, settings.partial_result_on_first_cancel); break; @@ -1491,6 +1505,7 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars &client_context->getSettingsRef(), &client_context->getClientInfo(), true, + {}, [&](const Progress & progress) { onProgress(progress); }); if (send_external_tables) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 07f4bf19f053..a08f71bfa906 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -5,8 +5,6 @@ #include #include #include -#include -#include #include #include #include @@ -16,6 +14,7 @@ #include #include #include +#include "Common/logger_useful.h" #include #include #include @@ -24,8 +23,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -85,6 +84,7 @@ Connection::~Connection() Connection::Connection(const String & host_, UInt16 port_, const String & default_database_, const String & user_, const String & password_, + const String & proto_send_chunked_, const String & proto_recv_chunked_, [[maybe_unused]] const SSHKey & ssh_private_key_, const String & jwt_, const String & quota_key_, @@ -95,6 +95,7 @@ Connection::Connection(const String & host_, UInt16 port_, Protocol::Secure secure_) : host(host_), port(port_), default_database(default_database_) , user(user_), password(password_) + , proto_send_chunked(proto_send_chunked_), proto_recv_chunked(proto_recv_chunked_) #if USE_SSH , ssh_private_key(ssh_private_key_) #endif @@ -211,10 +212,10 @@ void Connection::connect(const ConnectionTimeouts & timeouts) , tcp_keep_alive_timeout_in_sec); } - in = std::make_shared(*socket); + in = std::make_shared(*socket); in->setAsyncCallback(async_callback); - out = std::make_shared(*socket); + out = std::make_shared(*socket); out->setAsyncCallback(async_callback); connected = true; setDescription(); @@ -222,9 +223,61 @@ void Connection::connect(const ConnectionTimeouts & timeouts) sendHello(); receiveHello(timeouts.handshake_timeout); + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + /// Client side of chunked protocol negotiation. + /// Server advertises its protocol capabilities (separate for send and receive channels) by sending + /// in its 'Hello' response one of four types - chunked, notchunked, chunked_optional, notchunked_optional. + /// Not optional types are strict meaning that server only supports this type, optional means that + /// server prefer this type but capable to work in opposite. + /// Client selects which type it is going to communicate based on the settings from config or arguments, + /// and sends either "chunked" or "notchunked" protocol request in addendum section of handshake. + /// Client can detect if server's protocol capabilities are not compatible with client's settings (for example + /// server strictly requires chunked protocol but client's settings only allows notchunked protocol) - in such case + /// client should interrupt this connection. However if client continues with incompatible protocol type request, server + /// will send appropriate exception and disconnect client. + + auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) + { + bool chunked_srv = chunked_srv_str.starts_with("chunked"); + bool optional_srv = chunked_srv_str.ends_with("_optional"); + bool chunked_cl = chunked_cl_str.starts_with("chunked"); + bool optional_cl = chunked_cl_str.ends_with("_optional"); + + if (optional_srv) + return chunked_cl; + if (optional_cl) + return chunked_srv; + if (chunked_cl != chunked_srv) + throw NetException( + ErrorCodes::NETWORK_ERROR, + "Incompatible protocol: {} set to {}, server requires {}", + direction, + chunked_cl ? "chunked" : "notchunked", + chunked_srv ? "chunked" : "notchunked"); + + return chunked_srv; + }; + + proto_send_chunked = is_chunked(proto_recv_chunked_srv, proto_send_chunked, "send") ? "chunked" : "notchunked"; + proto_recv_chunked = is_chunked(proto_send_chunked_srv, proto_recv_chunked, "recv") ? "chunked" : "notchunked"; + } + else + { + if (proto_send_chunked == "chunked" || proto_recv_chunked == "chunked") + throw NetException( + ErrorCodes::NETWORK_ERROR, + "Incompatible protocol: server's version is too old and doesn't support chunked protocol while client settings require it."); + } + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) sendAddendum(); + if (proto_send_chunked == "chunked") + out->enableChunked(); + if (proto_recv_chunked == "chunked") + in->enableChunked(); + LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", server_name, server_version_major, server_version_minor, server_version_patch); } @@ -393,6 +446,16 @@ void Connection::sendAddendum() { if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_QUOTA_KEY) writeStringBinary(quota_key, *out); + + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + writeStringBinary(proto_send_chunked, *out); + writeStringBinary(proto_recv_chunked, *out); + } + + if (server_revision >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL) + writeVarUInt(DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION, *out); + out->next(); } @@ -463,6 +526,8 @@ void Connection::receiveHello(const Poco::Timespan & handshake_timeout) readVarUInt(server_version_major, *in); readVarUInt(server_version_minor, *in); readVarUInt(server_revision, *in); + if (server_revision >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL) + readVarUInt(server_parallel_replicas_protocol_version, *in); if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) readStringBinary(server_timezone, *in); if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME) @@ -472,6 +537,12 @@ void Connection::receiveHello(const Poco::Timespan & handshake_timeout) else server_version_patch = server_revision; + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + readStringBinary(proto_send_chunked_srv, *in); + readStringBinary(proto_recv_chunked_srv, *in); + } + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES) { UInt64 rules_size; @@ -611,6 +682,7 @@ bool Connection::ping(const ConnectionTimeouts & timeouts) UInt64 pong = 0; writeVarUInt(Protocol::Client::Ping, *out); + out->finishChunk(); out->next(); if (in->eof()) @@ -660,6 +732,7 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time writeVarUInt(Protocol::Client::TablesStatusRequest, *out); request.write(*out, server_revision); + out->finishChunk(); out->next(); UInt64 response_type = 0; @@ -685,6 +758,7 @@ void Connection::sendQuery( const Settings * settings, const ClientInfo * client_info, bool with_pending_data, + const std::vector & external_roles, std::function) { OpenTelemetry::SpanHolder span("Connection::sendQuery()", OpenTelemetry::SpanKind::CLIENT); @@ -758,6 +832,18 @@ void Connection::sendQuery( else writeStringBinary("" /* empty string is a marker of the end of settings */, *out); + String external_roles_str; + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_INTERSERVER_EXTERNALLY_GRANTED_ROLES) + { + WriteBufferFromString buffer(external_roles_str); + writeVectorBinary(external_roles, buffer); + buffer.finalize(); + + LOG_TRACE(log_wrapper.get(), "Sending external_roles with query: [{}] ({})", fmt::join(external_roles, ", "), external_roles.size()); + + writeStringBinary(external_roles_str, *out); + } + /// Interserver secret if (server_revision >= DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET) { @@ -778,6 +864,9 @@ void Connection::sendQuery( data += query; data += query_id; data += client_info->initial_user; + // Also for backwards compatibility + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_INTERSERVER_EXTERNALLY_GRANTED_ROLES) + data += external_roles_str; /// TODO: add source/target host/ip-address std::string hash = encodeSHA256(data); @@ -813,6 +902,8 @@ void Connection::sendQuery( block_profile_events_in.reset(); block_out.reset(); + out->finishChunk(); + /// Send empty block which means end of data. if (!with_pending_data) { @@ -829,6 +920,7 @@ void Connection::sendCancel() return; writeVarUInt(Protocol::Client::Cancel, *out); + out->finishChunk(); out->next(); } @@ -854,7 +946,10 @@ void Connection::sendData(const Block & block, const String & name, bool scalar) size_t prev_bytes = out->count(); block_out->write(block); - maybe_compressed_out->next(); + if (maybe_compressed_out != out) + maybe_compressed_out->next(); + if (!block) + out->finishChunk(); out->next(); if (throttler) @@ -865,6 +960,7 @@ void Connection::sendIgnoredPartUUIDs(const std::vector & uuids) { writeVarUInt(Protocol::Client::IgnoredPartUUIDs, *out); writeVectorBinary(uuids, *out); + out->finishChunk(); out->next(); } @@ -874,6 +970,7 @@ void Connection::sendReadTaskResponse(const String & response) writeVarUInt(Protocol::Client::ReadTaskResponse, *out); writeVarUInt(DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION, *out); writeStringBinary(response, *out); + out->finishChunk(); out->next(); } @@ -881,7 +978,8 @@ void Connection::sendReadTaskResponse(const String & response) void Connection::sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) { writeVarUInt(Protocol::Client::MergeTreeReadTaskResponse, *out); - response.serialize(*out); + response.serialize(*out, server_parallel_replicas_protocol_version); + out->finishChunk(); out->next(); } @@ -899,6 +997,8 @@ void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String copyData(input, *out); else copyData(input, *out, size); + + out->finishChunk(); out->next(); } @@ -927,6 +1027,8 @@ void Connection::sendScalarsData(Scalars & data) sendData(elem.second, elem.first, true /* scalar */); } + out->finishChunk(); + out_bytes = out->count() - out_bytes; maybe_compressed_out_bytes = maybe_compressed_out->count() - maybe_compressed_out_bytes; double elapsed = watch.elapsedSeconds(); @@ -1069,13 +1171,13 @@ std::optional Connection::getResolvedAddress() const bool Connection::poll(size_t timeout_microseconds) { - return static_cast(*in).poll(timeout_microseconds); + return in->poll(timeout_microseconds); } bool Connection::hasReadPendingData() const { - return last_input_packet_type.has_value() || static_cast(*in).hasPendingData(); + return last_input_packet_type.has_value() || in->hasBufferedData(); } @@ -1330,7 +1432,7 @@ ParallelReadRequest Connection::receiveParallelReadRequest() const InitialAllRangesAnnouncement Connection::receiveInitialParallelReadAnnouncement() const { - return InitialAllRangesAnnouncement::deserialize(*in); + return InitialAllRangesAnnouncement::deserialize(*in, server_parallel_replicas_protocol_version); } @@ -1349,6 +1451,8 @@ ServerConnectionPtr Connection::createConnection(const ConnectionParameters & pa parameters.default_database, parameters.user, parameters.password, + parameters.proto_send_chunked, + parameters.proto_recv_chunked, parameters.ssh_private_key, parameters.jwt, parameters.quota_key, diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 0f4b3e436df2..29939e5d5541 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -8,8 +8,8 @@ #include -#include -#include +#include +#include #include #include @@ -52,6 +52,7 @@ class Connection : public IServerConnection Connection(const String & host_, UInt16 port_, const String & default_database_, const String & user_, const String & password_, + const String & proto_send_chunked_, const String & proto_recv_chunked_, const SSHKey & ssh_private_key_, const String & jwt_, const String & quota_key_, @@ -107,6 +108,7 @@ class Connection : public IServerConnection const Settings * settings/* = nullptr */, const ClientInfo * client_info/* = nullptr */, bool with_pending_data/* = false */, + const std::vector & external_roles, std::function process_progress_callback) override; void sendCancel() override; @@ -170,6 +172,10 @@ class Connection : public IServerConnection String default_database; String user; String password; + String proto_send_chunked; + String proto_recv_chunked; + String proto_send_chunked_srv; + String proto_recv_chunked_srv; #if USE_SSH SSHKey ssh_private_key; #endif @@ -205,12 +211,13 @@ class Connection : public IServerConnection UInt64 server_version_minor = 0; UInt64 server_version_patch = 0; UInt64 server_revision = 0; + UInt64 server_parallel_replicas_protocol_version = 0; String server_timezone; String server_display_name; std::unique_ptr socket; - std::shared_ptr in; - std::shared_ptr out; + std::shared_ptr in; + std::shared_ptr out; std::optional last_input_packet_type; String query_id; diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 303bebc30d2d..4d0a9ffa08c2 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -107,6 +107,9 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati } } + proto_send_chunked = config.getString("proto_caps.send", "notchunked"); + proto_recv_chunked = config.getString("proto_caps.recv", "notchunked"); + quota_key = config.getString("quota_key", ""); /// By default compression is disabled if address looks like localhost. diff --git a/src/Client/ConnectionParameters.h b/src/Client/ConnectionParameters.h index c305c7813f2d..382bfe34a3d6 100644 --- a/src/Client/ConnectionParameters.h +++ b/src/Client/ConnectionParameters.h @@ -20,6 +20,8 @@ struct ConnectionParameters std::string default_database; std::string user; std::string password; + std::string proto_send_chunked = "notchunked"; + std::string proto_recv_chunked = "notchunked"; std::string quota_key; SSHKey ssh_private_key; std::string jwt; diff --git a/src/Client/ConnectionPool.cpp b/src/Client/ConnectionPool.cpp index ed2e7c3c7255..ab8ad08826cb 100644 --- a/src/Client/ConnectionPool.cpp +++ b/src/Client/ConnectionPool.cpp @@ -13,6 +13,8 @@ ConnectionPoolPtr ConnectionPoolFactory::get( String default_database, String user, String password, + String proto_send_chunked, + String proto_recv_chunked, String quota_key, String cluster, String cluster_secret, @@ -22,7 +24,7 @@ ConnectionPoolPtr ConnectionPoolFactory::get( Priority priority) { Key key{ - max_connections, host, port, default_database, user, password, quota_key, cluster, cluster_secret, client_name, compression, secure, priority}; + max_connections, host, port, default_database, user, password, proto_send_chunked, proto_recv_chunked, quota_key, cluster, cluster_secret, client_name, compression, secure, priority}; std::lock_guard lock(mutex); auto [it, inserted] = pools.emplace(key, ConnectionPoolPtr{}); @@ -39,6 +41,8 @@ ConnectionPoolPtr ConnectionPoolFactory::get( default_database, user, password, + proto_send_chunked, + proto_recv_chunked, quota_key, cluster, cluster_secret, diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index 0fcb3c4e7e15..219548b62a04 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -73,6 +73,8 @@ class ConnectionPool : public IConnectionPool, private PoolBase const String & default_database_, const String & user_, const String & password_, + const String & proto_send_chunked_, + const String & proto_recv_chunked_, const String & quota_key_, const String & cluster_, const String & cluster_secret_, @@ -85,6 +87,8 @@ class ConnectionPool : public IConnectionPool, private PoolBase , default_database(default_database_) , user(user_) , password(password_) + , proto_send_chunked(proto_send_chunked_) + , proto_recv_chunked(proto_recv_chunked_) , quota_key(quota_key_) , cluster(cluster_) , cluster_secret(cluster_secret_) @@ -116,7 +120,9 @@ class ConnectionPool : public IConnectionPool, private PoolBase { return std::make_shared( host, port, - default_database, user, password, SSHKey(), /*jwt*/ "", quota_key, + default_database, user, password, + proto_send_chunked, proto_recv_chunked, + SSHKey(), /*jwt*/ "", quota_key, cluster, cluster_secret, client_name, compression, secure); } @@ -125,6 +131,8 @@ class ConnectionPool : public IConnectionPool, private PoolBase String default_database; String user; String password; + String proto_send_chunked; + String proto_recv_chunked; String quota_key; /// For inter-server authorization @@ -150,6 +158,8 @@ class ConnectionPoolFactory final : private boost::noncopyable String default_database; String user; String password; + String proto_send_chunked; + String proto_recv_chunked; String quota_key; String cluster; String cluster_secret; @@ -173,6 +183,8 @@ class ConnectionPoolFactory final : private boost::noncopyable String default_database, String user, String password, + String proto_send_chunked, + String proto_recv_chunked, String quota_key, String cluster, String cluster_secret, @@ -190,6 +202,7 @@ inline bool operator==(const ConnectionPoolFactory::Key & lhs, const ConnectionP { return lhs.max_connections == rhs.max_connections && lhs.host == rhs.host && lhs.port == rhs.port && lhs.default_database == rhs.default_database && lhs.user == rhs.user && lhs.password == rhs.password + && lhs.proto_send_chunked == rhs.proto_send_chunked && lhs.proto_recv_chunked == rhs.proto_recv_chunked && lhs.quota_key == rhs.quota_key && lhs.cluster == rhs.cluster && lhs.cluster_secret == rhs.cluster_secret && lhs.client_name == rhs.client_name && lhs.compression == rhs.compression && lhs.secure == rhs.secure && lhs.priority == rhs.priority; diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index dd8348ea04f1..39d92a075b0d 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -146,7 +146,8 @@ void HedgedConnections::sendQuery( const String & query_id, UInt64 stage, ClientInfo & client_info, - bool with_pending_data) + bool with_pending_data, + const std::vector & external_roles) { std::lock_guard lock(cancel_mutex); @@ -173,7 +174,7 @@ void HedgedConnections::sendQuery( hedged_connections_factory.skipReplicasWithTwoLevelAggregationIncompatibility(); } - auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaState & replica) + auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data, external_roles](ReplicaState & replica) { Settings modified_settings = settings; @@ -203,7 +204,8 @@ void HedgedConnections::sendQuery( modified_settings.set("allow_experimental_analyzer", static_cast(modified_settings.allow_experimental_analyzer)); replica.connection->sendQuery( - timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, {}); + timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, external_roles, {}); + replica.change_replica_timeout.setRelative(timeouts.receive_data_timeout); replica.packet_receiver->setTimeout(hedged_connections_factory.getConnectionTimeouts().receive_timeout); }; diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index 7f538804e5a6..e64f17658d8b 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -90,7 +90,8 @@ class HedgedConnections : public IConnections const String & query_id, UInt64 stage, ClientInfo & client_info, - bool with_pending_data) override; + bool with_pending_data, + const std::vector & external_roles) override; void sendReadTaskResponse(const String &) override { diff --git a/src/Client/IConnections.h b/src/Client/IConnections.h index 09211de53b0a..a521fdd8b004 100644 --- a/src/Client/IConnections.h +++ b/src/Client/IConnections.h @@ -23,7 +23,8 @@ class IConnections : boost::noncopyable const String & query_id, UInt64 stage, ClientInfo & client_info, - bool with_pending_data) = 0; + bool with_pending_data, + const std::vector & external_roles) = 0; virtual void sendReadTaskResponse(const String &) = 0; virtual void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) = 0; diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 6ab4234bca24..f6f62eb90d8f 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -102,6 +102,7 @@ class IServerConnection : boost::noncopyable const Settings * settings, const ClientInfo * client_info, bool with_pending_data, + const std::vector & external_roles, std::function process_progress_callback) = 0; virtual void sendCancel() = 0; diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 7595a29912b2..0a4e90ecaaa4 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -95,6 +95,7 @@ void LocalConnection::sendQuery( const Settings *, const ClientInfo * client_info, bool, + const std::vector & /*external_roles*/, std::function process_progress_callback) { /// Last query may not have been finished or cancelled due to exception on client side. diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index b424c5b5aa3d..ae0e93e40999 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -114,6 +114,7 @@ class LocalConnection : public IServerConnection, WithContext const Settings * settings/* = nullptr */, const ClientInfo * client_info/* = nullptr */, bool with_pending_data/* = false */, + const std::vector & external_roles, std::function process_progress_callback) override; void sendCancel() override; diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 244eccf1ed96..484df6154c2f 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -118,7 +118,8 @@ void MultiplexedConnections::sendQuery( const String & query_id, UInt64 stage, ClientInfo & client_info, - bool with_pending_data) + bool with_pending_data, + const std::vector & external_roles) { std::lock_guard lock(cancel_mutex); @@ -171,14 +172,14 @@ void MultiplexedConnections::sendQuery( modified_settings.parallel_replica_offset = i; replica_states[i].connection->sendQuery( - timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, {}); + timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, external_roles, {}); } } else { /// Use single replica. replica_states[0].connection->sendQuery( - timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, {}); + timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, external_roles, {}); } sent_query = true; diff --git a/src/Client/MultiplexedConnections.h b/src/Client/MultiplexedConnections.h index dec32e52d4f8..4b308dca02e9 100644 --- a/src/Client/MultiplexedConnections.h +++ b/src/Client/MultiplexedConnections.h @@ -36,7 +36,8 @@ class MultiplexedConnections final : public IConnections const String & query_id, UInt64 stage, ClientInfo & client_info, - bool with_pending_data) override; + bool with_pending_data, + const std::vector & external_roles) override; void sendReadTaskResponse(const String &) override; void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) override; diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index affd620f83ab..afb98b7d95c1 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -164,7 +164,7 @@ void Suggest::load(IServerConnection & connection, void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & timeouts, const std::string & query, const ClientInfo & client_info) { connection.sendQuery( - timeouts, query, {} /* query_parameters */, "" /* query_id */, QueryProcessingStage::Complete, nullptr, &client_info, false, {}); + timeouts, query, {} /* query_parameters */, "" /* query_id */, QueryProcessingStage::Complete, nullptr, &client_info, false, {} /* external_roles*/, {}); while (true) { diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 49c9ae2c27c4..894a8667f28a 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -365,10 +365,13 @@ The server successfully detected this situation and will download merged part fr \ M(ParallelReplicasHandleRequestMicroseconds, "Time spent processing requests for marks from replicas") \ M(ParallelReplicasHandleAnnouncementMicroseconds, "Time spent processing replicas announcements") \ + M(ParallelReplicasAnnouncementMicroseconds, "Time spent to send an announcement") \ + M(ParallelReplicasReadRequestMicroseconds, "Time spent for read requests") \ \ 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(ParallelReplicasReadMarks, "How many marks were read by the given replica") \ \ M(ParallelReplicasStealingByHashMicroseconds, "Time spent collecting segments meant for stealing by hash") \ M(ParallelReplicasProcessingPartsMicroseconds, "Time spent processing data parts") \ @@ -522,6 +525,7 @@ The server successfully detected this situation and will download merged part fr M(CachedReadBufferReadFromCacheMicroseconds, "Time reading from filesystem cache") \ M(CachedReadBufferReadFromSourceBytes, "Bytes read from filesystem cache source (from remote fs, etc)") \ M(CachedReadBufferReadFromCacheBytes, "Bytes read from filesystem cache") \ + M(CachedReadBufferPredownloadedBytes, "Bytes read from filesystem cache source. Cache segments are read from left to right as a whole, it might be that we need to predownload some part of the segment irrelevant for the current task just to get to the needed data") \ M(CachedReadBufferCacheWriteBytes, "Bytes written from source (remote fs, etc) to filesystem cache") \ M(CachedReadBufferCacheWriteMicroseconds, "Time spent writing data into filesystem cache") \ M(CachedReadBufferCreateBufferMicroseconds, "Prepare buffer time") \ diff --git a/src/Core/ParallelReplicasMode.h b/src/Core/ParallelReplicasMode.h new file mode 100644 index 000000000000..dd0d9c05f0de --- /dev/null +++ b/src/Core/ParallelReplicasMode.h @@ -0,0 +1,25 @@ +#pragma once + +#include + +namespace DB +{ + +/// Deprecated. Used only to define a type for the obsolete setting. +enum class ParallelReplicasCustomKeyFilterType : UInt8 +{ + DEFAULT, + RANGE, +}; + +enum class ParallelReplicasMode : UInt8 +{ + AUTO = 0, + READ_TASKS = 1, + CUSTOM_KEY_SAMPLING = 2, + CUSTOM_KEY_RANGE = 3, + SAMPLING_KEY = 4, +}; + + +} diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 02d54221ed3e..cd89ca013eea 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -33,7 +33,9 @@ static constexpr auto DBMS_MIN_REVISION_WITH_AGGREGATE_FUNCTIONS_VERSIONING = 54 static constexpr auto DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION = 1; -static constexpr auto DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION = 3; +static constexpr auto DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION = 3; +static constexpr auto DBMS_PARALLEL_REPLICAS_MIN_VERSION_WITH_MARK_SEGMENT_SIZE_FIELD = 4; +static constexpr auto DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION = 4; static constexpr auto DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS = 54453; static constexpr auto DBMS_MERGE_TREE_PART_INFO_VERSION = 1; @@ -83,6 +85,14 @@ static constexpr auto DBMS_MIN_REVISION_WITH_SYSTEM_KEYWORDS_TABLE = 54468; static constexpr auto DBMS_MIN_REVISION_WITH_ROWS_BEFORE_AGGREGATION = 54469; +/// Packets size header +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS = 54470; + +static constexpr auto DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL = 54471; + +/// Push externally granted roles to other nodes +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_INTERSERVER_EXTERNALLY_GRANTED_ROLES = 54472; + /// Version of ClickHouse TCP protocol. /// /// Should be incremented manually on protocol changes. @@ -90,6 +100,6 @@ static constexpr auto DBMS_MIN_REVISION_WITH_ROWS_BEFORE_AGGREGATION = 54469; /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54469; +static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54472; } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3edb6ff6c1b8..b82733f2a44c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -205,6 +205,7 @@ class IColumn; M(Bool, group_by_use_nulls, false, "Treat columns mentioned in ROLLUP, CUBE or GROUPING SETS as Nullable", 0) \ \ M(NonZeroUInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled. Should be always greater than 0", 0) \ + M(ParallelReplicasMode, parallel_replicas_mode, ParallelReplicasMode::READ_TASKS, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ \ M(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) \ \ @@ -898,6 +899,7 @@ class IColumn; M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \ M(Bool, restore_replace_external_engines_to_null, false, "Replace all the external table engines to Null on restore. Useful for testing purposes", 0) \ M(Bool, restore_replace_external_table_functions_to_null, false, "Replace all table functions to Null on restore. Useful for testing purposes", 0) \ + M(Bool, push_external_roles_in_interserver_queries, true, "Enable pushing user roles from originator to other nodes while performing a query.", 0) \ \ \ /* ###################################### */ \ @@ -930,18 +932,18 @@ class IColumn; M(UInt64, parallel_replicas_count, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing.", 0) \ M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \ M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \ - M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ M(UInt64, parallel_replicas_custom_key_range_lower, 0, "Lower bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ M(UInt64, parallel_replicas_custom_key_range_upper, 0, "Upper bound for the universe that the parallel replicas custom range filter is calculated over. A value of 0 disables the upper bound, setting it to the max value of the custom key expression", 0) \ M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ - M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ + M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use up to `max_parallel_replicas` the number of replicas from each shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) ALIAS(enable_parallel_replicas)\ M(Bool, parallel_replicas_allow_in_with_subquery, true, "If true, subquery for IN will be executed on every follower replica.", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ 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(Bool, parallel_replicas_prefer_local_join, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN.", 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(UInt64, parallel_replicas_mark_segment_size, 0, "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. Value should be in range [128; 16384]", 0) \ M(Bool, allow_archive_path_syntax, true, "File/S3 engines/table function will parse paths with '::' as ' :: ' if archive has correct extension", 0) \ + M(Bool, parallel_replicas_local_plan, false, "Build local plan for local replica", 0) \ \ M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \ M(Bool, allow_experimental_full_text_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \ @@ -1035,6 +1037,7 @@ class IColumn; MAKE_OBSOLETE(M, UInt64, restore_threads, 16) \ MAKE_OBSOLETE(M, Bool, optimize_duplicate_order_by_and_distinct, false) \ MAKE_OBSOLETE(M, UInt64, parallel_replicas_min_number_of_granules_to_enable, 0) \ + MAKE_OBSOLETE(M, ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT) \ MAKE_OBSOLETE(M, Bool, query_plan_optimize_projection, true) \ MAKE_OBSOLETE(M, Bool, query_cache_store_results_of_queries_with_nondeterministic_functions, false) \ MAKE_OBSOLETE(M, Bool, allow_experimental_annoy_index, false) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index ee2cce99ff2b..4b3d5f60a8ef 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -63,6 +63,7 @@ static std::initializer_list #include #include +#include #include #include #include @@ -299,12 +300,6 @@ enum class Dialect : uint8_t DECLARE_SETTING_ENUM(Dialect) -enum class ParallelReplicasCustomKeyFilterType : uint8_t -{ - DEFAULT, - RANGE, -}; - DECLARE_SETTING_ENUM(ParallelReplicasCustomKeyFilterType) enum class LightweightMutationProjectionMode : uint8_t @@ -324,6 +319,8 @@ enum class DeduplicateMergeProjectionMode : uint8_t DECLARE_SETTING_ENUM(DeduplicateMergeProjectionMode) +DECLARE_SETTING_ENUM(ParallelReplicasMode) + DECLARE_SETTING_ENUM(LocalFSReadMethod) enum class ObjectStorageQueueMode : uint8_t diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index bf16f315ddf4..b36d53a61596 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -51,6 +51,8 @@ namespace configuration.db, configuration.user, configuration.password, + configuration.proto_send_chunked, + configuration.proto_recv_chunked, configuration.quota_key, "", /* cluster */ "", /* cluster_secret */ @@ -222,7 +224,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) { validateNamedCollection( *named_collection, {}, ValidateKeysMultiset{ - "secure", "host", "hostname", "port", "user", "username", "password", "quota_key", "name", + "secure", "host", "hostname", "port", "user", "username", "password", "proto_send_chunked", "proto_recv_chunked", "quota_key", "name", "db", "database", "table","query", "where", "invalidate_query", "update_field", "update_lag"}); const auto secure = named_collection->getOrDefault("secure", false); @@ -234,6 +236,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = named_collection->getAnyOrDefault({"user", "username"}, "default"), .password = named_collection->getOrDefault("password", ""), + .proto_send_chunked = named_collection->getOrDefault("proto_send_chunked", "notchunked"), + .proto_recv_chunked = named_collection->getOrDefault("proto_recv_chunked", "notchunked"), .quota_key = named_collection->getOrDefault("quota_key", ""), .db = named_collection->getAnyOrDefault({"db", "database"}, default_database), .table = named_collection->getOrDefault("table", ""), @@ -258,6 +262,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = config.getString(settings_config_prefix + ".user", "default"), .password = config.getString(settings_config_prefix + ".password", ""), + .proto_send_chunked = config.getString(settings_config_prefix + ".proto_caps.send", "notchunked"), + .proto_recv_chunked = config.getString(settings_config_prefix + ".proto_caps.recv", "notchunked"), .quota_key = config.getString(settings_config_prefix + ".quota_key", ""), .db = config.getString(settings_config_prefix + ".db", default_database), .table = config.getString(settings_config_prefix + ".table", ""), diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index 3357514eab2b..faf9e5f80090 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -23,6 +23,8 @@ class ClickHouseDictionarySource final : public IDictionarySource const std::string host; const std::string user; const std::string password; + const std::string proto_send_chunked; + const std::string proto_recv_chunked; const std::string quota_key; const std::string db; const std::string table; diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index b471f3fc58f9..6363c40a9ad2 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -28,6 +28,7 @@ extern const Event CachedReadBufferReadFromCacheMicroseconds; extern const Event CachedReadBufferCacheWriteMicroseconds; extern const Event CachedReadBufferReadFromSourceBytes; extern const Event CachedReadBufferReadFromCacheBytes; +extern const Event CachedReadBufferPredownloadedBytes; extern const Event CachedReadBufferCacheWriteBytes; extern const Event CachedReadBufferCreateBufferMicroseconds; @@ -644,6 +645,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment) size_t current_predownload_size = std::min(current_impl_buffer_size, bytes_to_predownload); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromSourceBytes, current_impl_buffer_size); + ProfileEvents::increment(ProfileEvents::CachedReadBufferPredownloadedBytes, current_impl_buffer_size); bool continue_predownload = file_segment.reserve( current_predownload_size, settings.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds); diff --git a/src/IO/NetUtils.h b/src/IO/NetUtils.h new file mode 100644 index 000000000000..12f09524ae7a --- /dev/null +++ b/src/IO/NetUtils.h @@ -0,0 +1,58 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +constexpr T netToHost(T value) noexcept +{ + if constexpr (std::endian::native != std::endian::big) + return std::byteswap(value); + return value; +} + +template +constexpr T hostToNet(T value) noexcept +{ + if constexpr (std::endian::native != std::endian::big) + return std::byteswap(value); + return value; +} + +template +constexpr T toLittleEndian(T value) noexcept +{ + if constexpr (std::endian::native == std::endian::big) + return std::byteswap(value); + return value; +} + +template +constexpr T toBigEndian(T value) noexcept +{ + if constexpr (std::endian::native != std::endian::big) + return std::byteswap(value); + return value; +} + +template +constexpr T fromLittleEndian(T value) noexcept +{ + if constexpr (std::endian::native == std::endian::big) + return std::byteswap(value); + return value; +} + +template +constexpr T fromBigEndian(T value) noexcept +{ + if constexpr (std::endian::native != std::endian::big) + return std::byteswap(value); + return value; +} + +} diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 26cdee4140c8..bbf9f96404fc 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -32,7 +32,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -bool ReadBufferFromPocoSocket::nextImpl() +ssize_t ReadBufferFromPocoSocketBase::socketReceiveBytesImpl(char * ptr, size_t size) { ssize_t bytes_read = 0; Stopwatch watch; @@ -43,14 +43,11 @@ bool ReadBufferFromPocoSocket::nextImpl() ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read); }); + CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkReceive); + /// Add more details to exceptions. try { - CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkReceive); - - if (internal_buffer.size() > INT_MAX) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); - /// If async_callback is specified, set socket to non-blocking mode /// and try to read data from it, if socket is not ready for reading, /// run async_callback and try again later. @@ -61,7 +58,7 @@ bool ReadBufferFromPocoSocket::nextImpl() socket.setBlocking(false); SCOPE_EXIT(socket.setBlocking(true)); bool secure = socket.secure(); - bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + bytes_read = socket.impl()->receiveBytes(ptr, static_cast(size)); /// Check EAGAIN and ERR_SSL_WANT_READ/ERR_SSL_WANT_WRITE for secure socket (reading from secure socket can write too). while (bytes_read < 0 && (errno == EAGAIN || (secure && (checkSSLWantRead(bytes_read) || checkSSLWantWrite(bytes_read))))) @@ -73,12 +70,12 @@ bool ReadBufferFromPocoSocket::nextImpl() async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); /// Try to read again. - bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + bytes_read = socket.impl()->receiveBytes(ptr, static_cast(size)); } } else { - bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + bytes_read = socket.impl()->receiveBytes(ptr, static_cast(size)); } } catch (const Poco::Net::NetException & e) @@ -99,6 +96,16 @@ bool ReadBufferFromPocoSocket::nextImpl() if (bytes_read < 0) throw NetException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot read from socket (peer: {}, local: {})", peer_address.toString(), socket.address().toString()); + return bytes_read; +} + +bool ReadBufferFromPocoSocketBase::nextImpl() +{ + if (internal_buffer.size() > INT_MAX) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); + + ssize_t bytes_read = socketReceiveBytesImpl(internal_buffer.begin(), internal_buffer.size()); + if (read_event != ProfileEvents::end()) ProfileEvents::increment(read_event, bytes_read); @@ -110,7 +117,7 @@ bool ReadBufferFromPocoSocket::nextImpl() return true; } -ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) +ReadBufferFromPocoSocketBase::ReadBufferFromPocoSocketBase(Poco::Net::Socket & socket_, size_t buf_size) : BufferWithOwnMemory(buf_size) , socket(socket_) , peer_address(socket.peerAddress()) @@ -119,19 +126,22 @@ ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, { } -ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) - : ReadBufferFromPocoSocket(socket_, buf_size) +ReadBufferFromPocoSocketBase::ReadBufferFromPocoSocketBase(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) + : ReadBufferFromPocoSocketBase(socket_, buf_size) { read_event = read_event_; } -bool ReadBufferFromPocoSocket::poll(size_t timeout_microseconds) const +bool ReadBufferFromPocoSocketBase::poll(size_t timeout_microseconds) const { - if (available()) + /// For secure socket it is important to check if any remaining data available in underlying decryption buffer - + /// read always retrieves the whole encrypted frame from the wire and puts it into underlying buffer while returning only requested size - + /// further poll() can block though there is still data to read in the underlying decryption buffer. + if (available() || socket.impl()->available()) return true; Stopwatch watch; - bool res = socket.poll(timeout_microseconds, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR); + bool res = socket.impl()->poll(timeout_microseconds, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR); ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); return res; } diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index 76156612764b..912388adaac6 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -9,7 +9,7 @@ namespace DB { /// Works with the ready Poco::Net::Socket. Blocking operations. -class ReadBufferFromPocoSocket : public BufferWithOwnMemory +class ReadBufferFromPocoSocketBase : public BufferWithOwnMemory { protected: Poco::Net::Socket & socket; @@ -25,16 +25,29 @@ class ReadBufferFromPocoSocket : public BufferWithOwnMemory bool nextImpl() override; public: - explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); - explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + explicit ReadBufferFromPocoSocketBase(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + explicit ReadBufferFromPocoSocketBase(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); bool poll(size_t timeout_microseconds) const; void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); } + ssize_t socketReceiveBytesImpl(char * ptr, size_t size); + private: AsyncCallback async_callback; std::string socket_description; }; +class ReadBufferFromPocoSocket : public ReadBufferFromPocoSocketBase +{ +public: + explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) + : ReadBufferFromPocoSocketBase(socket_, buf_size) + {} + explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) + : ReadBufferFromPocoSocketBase(socket_, read_event_, buf_size) + {} +}; + } diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp new file mode 100644 index 000000000000..4a1e3732a557 --- /dev/null +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -0,0 +1,166 @@ +#include +#include +#include + + +namespace DB::ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace DB +{ + +ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size) + : ReadBufferFromPocoSocketChunked(socket_, ProfileEvents::end(), buf_size) +{} + +ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) + : ReadBufferFromPocoSocketBase( + socket_, read_event_, + std::min(buf_size, static_cast(std::numeric_limits::max()))), + our_address(socket_.address()), log(getLogger("Protocol")) +{} + +void ReadBufferFromPocoSocketChunked::enableChunked() +{ + if (chunked) + return; + chunked = 1; + data_end = buffer().end(); + /// Resize working buffer so any next read will call nextImpl + working_buffer.resize(offset()); + chunk_left = 0; + next_chunk = 0; +} + +bool ReadBufferFromPocoSocketChunked::hasBufferedData() const +{ + if (available()) + return true; + + return chunked && (static_cast(data_end - working_buffer.end()) > sizeof(next_chunk)); +} + +bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) const +{ + if (chunked) + if (available() || static_cast(data_end - working_buffer.end()) > sizeof(next_chunk)) + return true; + + return ReadBufferFromPocoSocketBase::poll(timeout_microseconds); +} + + +bool ReadBufferFromPocoSocketChunked::loadNextChunk(Position c_pos, bool cont) +{ + auto buffered = std::min(static_cast(data_end - c_pos), sizeof(next_chunk)); + + if (buffered) + std::memcpy(&next_chunk, c_pos, buffered); + if (buffered < sizeof(next_chunk)) + if (socketReceiveBytesImpl(reinterpret_cast(&next_chunk) + buffered, sizeof(next_chunk) - buffered) < static_cast(sizeof(next_chunk) - buffered)) + return false; + next_chunk = fromLittleEndian(next_chunk); + + if (next_chunk) + { + if (cont) + LOG_TEST(log, "{} <- {} Chunk receive continued. Size {}", ourAddress().toString(), peerAddress().toString(), next_chunk); + } + else + LOG_TEST(log, "{} <- {} Chunk receive ended.", ourAddress().toString(), peerAddress().toString()); + + return true; +} + +bool ReadBufferFromPocoSocketChunked::processChunkLeft(Position c_pos) +{ + if (data_end - c_pos < chunk_left) + { + working_buffer.resize(data_end - buffer().begin()); + nextimpl_working_buffer_offset = c_pos - buffer().begin(); + chunk_left -= (data_end - c_pos); + return true; + } + + nextimpl_working_buffer_offset = c_pos - buffer().begin(); + working_buffer.resize(nextimpl_working_buffer_offset + chunk_left); + + c_pos += chunk_left; + + if (!loadNextChunk(c_pos, true)) + return false; + + chunk_left = 0; + return true; +} + + +bool ReadBufferFromPocoSocketChunked::nextImpl() +{ + if (!chunked) + return ReadBufferFromPocoSocketBase::nextImpl(); + + auto * c_pos = pos; + + if (chunk_left == 0) + { + if (next_chunk == 0) + { + if (chunked == 1) + chunked = 2; // first chunked block - no end marker + else + c_pos = pos + sizeof(next_chunk); // bypass chunk end marker + + if (c_pos > data_end) + c_pos = data_end; + + if (!loadNextChunk(c_pos)) + return false; + + chunk_left = next_chunk; + next_chunk = 0; + + if (chunk_left == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: empty chunk received"); + + c_pos += sizeof(next_chunk); + + if (c_pos >= data_end) + { + if (!ReadBufferFromPocoSocketBase::nextImpl()) + return false; + data_end = buffer().end(); + c_pos = buffer().begin(); + } + + LOG_TEST(log, "{} <- {} Chunk receive started. Message {}, size {}", ourAddress().toString(), peerAddress().toString(), static_cast(*c_pos), chunk_left); + } + else + { + c_pos += sizeof(next_chunk); + if (c_pos >= data_end) + { + if (!ReadBufferFromPocoSocketBase::nextImpl()) + return false; + data_end = buffer().end(); + c_pos = buffer().begin(); + } + + chunk_left = next_chunk; + next_chunk = 0; + } + } + else + { + if (!ReadBufferFromPocoSocketBase::nextImpl()) + return false; + data_end = buffer().end(); + c_pos = buffer().begin(); + } + + return processChunkLeft(c_pos); +} + +} diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h new file mode 100644 index 000000000000..8bc4024b978a --- /dev/null +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -0,0 +1,109 @@ +#pragma once + +#include +#include + +/* + +Handshake +============= + | 'Hello' type + | handshake exchange + | chunked protocol negotiation + +============= + + +Basic chunk: + +============= +Chunk begins | 0x12345678 chunk size, 4 bytes little endian + +------------- + | Packet type always follows beginning of the chunk + | packet data + +------------- +Chunk ends | 0x00000000 4 zero bytes + +============= + + +Datastream chunk: + +============= +Chunk begins | 0x12345678 + +------------- + | Packet type + | packet data + +------------- + | Packet type + | packet data + +------------- +...arbitrary number ..... +of packets... ..... + +------------- + | Packet type + | packet data + +------------- +Chunk ends | 0x00000000 + +============= + + +Multipart chunk: + +============= +Chunk begins | 0x12345678 chunk part size, 4 bytes little endian + +------------- + | Packet type + | packet data + +------------- + | Packet type + | (partial) packet data + +============= +Chunk continues | 0x12345678 chunk next part size, 4 bytes little endian + +============= + | possibly previous packet's data + +------------- + | Packet type + | packet data + +------------- +...arbitrary number ..... +of chunk parts... ..... + +------------- + | Packet type + | packet data + +------------- +Chunk ends | 0x00000000 + +============= + +*/ + +namespace DB +{ + +class ReadBufferFromPocoSocketChunked: public ReadBufferFromPocoSocketBase +{ +public: + using ReadBufferFromPocoSocketBase::setAsyncCallback; + + explicit ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + explicit ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + + void enableChunked(); + + bool hasBufferedData() const; + + bool poll(size_t timeout_microseconds) const; + + Poco::Net::SocketAddress peerAddress() { return peer_address; } + Poco::Net::SocketAddress ourAddress() { return our_address; } + +protected: + bool loadNextChunk(Position c_pos, bool cont = false); + bool processChunkLeft(Position c_pos); + bool nextImpl() override; + + Poco::Net::SocketAddress our_address; + +private: + LoggerPtr log; + Position data_end = nullptr; // end position of data in the internal_buffer + UInt32 chunk_left = 0; // chunk left to read from socket + UInt32 next_chunk = 0; // size of the next cnunk + UInt8 chunked = 0; // 0 - disabled; 1 - started; 2 - enabled; +}; + +} diff --git a/src/IO/WriteBuffer.h b/src/IO/WriteBuffer.h index 4759f96a2352..84b1079b8247 100644 --- a/src/IO/WriteBuffer.h +++ b/src/IO/WriteBuffer.h @@ -64,7 +64,8 @@ class WriteBuffer : public BufferBase } bytes += bytes_in_buffer; - pos = working_buffer.begin(); + pos = working_buffer.begin() + nextimpl_working_buffer_offset; + nextimpl_working_buffer_offset = 0; } /// Calling finalize() in the destructor of derived classes is a bad practice. @@ -164,6 +165,11 @@ class WriteBuffer : public BufferBase bool finalized = false; bool canceled = false; + /// The number of bytes to preserve from the initial position of `working_buffer` + /// buffer. Apparently this is an additional out-parameter for nextImpl(), + /// not a real field. + size_t nextimpl_working_buffer_offset = 0; + private: /** Write the data in the buffer (from the beginning of the buffer to the current position). * Throw an exception if something is wrong. diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 5ed4dbdc7877..ffb38a384a06 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -183,6 +183,7 @@ WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_ , socket(socket_) , peer_address(socket.peerAddress()) , our_address(socket.address()) + , write_event(ProfileEvents::end()) , socket_description("socket (" + peer_address.toString() + ")") { } diff --git a/src/IO/WriteBufferFromPocoSocketChunked.cpp b/src/IO/WriteBufferFromPocoSocketChunked.cpp new file mode 100644 index 000000000000..9da46ee2d10f --- /dev/null +++ b/src/IO/WriteBufferFromPocoSocketChunked.cpp @@ -0,0 +1,210 @@ +#include +#include +#include + + +namespace +{ + +template +void setValue(T * typed_ptr, std::type_identity_t val) +{ + memcpy(static_cast(typed_ptr), &val, sizeof(T)); +} + +} + +namespace DB +{ + +WriteBufferFromPocoSocketChunked::WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size) + : WriteBufferFromPocoSocketChunked(socket_, ProfileEvents::end(), buf_size) +{} + +WriteBufferFromPocoSocketChunked::WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size) + : WriteBufferFromPocoSocket( + socket_, write_event_, + std::clamp(buf_size, sizeof(*chunk_size_ptr) + 1, static_cast(std::numeric_limits>::max()))), + log(getLogger("Protocol")) +{} + +void WriteBufferFromPocoSocketChunked::enableChunked() +{ + chunked = true; + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); + /// Pretend finishChunk() was just called to prevent sending empty chunk if finishChunk() called immediately + last_finish_chunk = chunk_size_ptr; +} + +void WriteBufferFromPocoSocketChunked::finishChunk() +{ + if (!chunked) + return; + + if (pos <= reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr)) + { + /// Prevent duplicate finish chunk (and finish chunk right after enableChunked()) + if (chunk_size_ptr == last_finish_chunk) + return; + + /// If current chunk is empty it means we are finishing a chunk previously sent by next(), + /// we want to convert current chunk header into end-of-chunk marker and initialize next chunk. + /// We don't need to worry about if it's the end of the buffer because next() always sends the whole buffer + /// so it should be a beginning of the buffer. + + chassert(reinterpret_cast(chunk_size_ptr) == working_buffer.begin()); + + setValue(chunk_size_ptr, 0); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); + + last_finish_chunk = chunk_size_ptr; + + return; + } + + /// Previously finished chunk wasn't sent yet + if (last_finish_chunk == chunk_size_ptr) + { + chunk_started = false; + LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); + } + + /// Fill up current chunk size + setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); + + if (!chunk_started) + LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", + ourAddress().toString(), peerAddress().toString(), + static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), + *chunk_size_ptr); + else + { + chunk_started = false; + LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); + } + + LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); + + if (available() < sizeof(*chunk_size_ptr)) + { + finishing = available(); + pos += available(); + chunk_size_ptr = reinterpret_cast(pos); + last_finish_chunk = chunk_size_ptr; + return; + } + + /// Buffer end-of-chunk + setValue(reinterpret_cast(pos), 0); + pos += sizeof(*chunk_size_ptr); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); + + last_finish_chunk = chunk_size_ptr; +} + +WriteBufferFromPocoSocketChunked::~WriteBufferFromPocoSocketChunked() +{ + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +void WriteBufferFromPocoSocketChunked::nextImpl() +{ + if (!chunked) + { + WriteBufferFromPocoSocket::nextImpl(); + return; + } + + /// next() after finishChunk at the end of the buffer + if (finishing < sizeof(*chunk_size_ptr)) + { + pos -= finishing; + /// Send current chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Send end-of-chunk directly + UInt32 s = 0; + socketSendBytes(reinterpret_cast(&s), sizeof(s)); + + finishing = sizeof(*chunk_size_ptr); + + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = chunk_size_ptr; + + return; + } + + /// Prevent sending empty chunk + if (offset() == sizeof(*chunk_size_ptr)) + { + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + return; + } + + /// Finish chunk at the end of the buffer + if (working_buffer.end() - reinterpret_cast(chunk_size_ptr) <= static_cast(sizeof(*chunk_size_ptr))) + { + pos = reinterpret_cast(chunk_size_ptr); + /// Send current chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = nullptr; + + return; + } + + bool initialize_last_finish_chunk = false; + if (pos - reinterpret_cast(chunk_size_ptr) == sizeof(*chunk_size_ptr)) // next() after finishChunk + { + pos -= sizeof(*chunk_size_ptr); + initialize_last_finish_chunk = true; + } + else // fill up current chunk size + { + setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); + if (!chunk_started) + { + chunk_started = true; + LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", + ourAddress().toString(), peerAddress().toString(), + static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), + *chunk_size_ptr); + } + else + LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); + } + /// Send current chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = initialize_last_finish_chunk ? chunk_size_ptr : nullptr; +} + +void WriteBufferFromPocoSocketChunked::finalizeImpl() +{ + if (chunked && offset() == sizeof(*chunk_size_ptr)) + pos -= sizeof(*chunk_size_ptr); + WriteBufferFromPocoSocket::finalizeImpl(); +} + +} diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h new file mode 100644 index 000000000000..13a277e3bfbd --- /dev/null +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class WriteBufferFromPocoSocketChunked: public WriteBufferFromPocoSocket +{ +public: + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + + void enableChunked(); + void finishChunk(); + ~WriteBufferFromPocoSocketChunked() override; + +protected: + void nextImpl() override; + void finalizeImpl() override; + Poco::Net::SocketAddress peerAddress() const { return peer_address; } + Poco::Net::SocketAddress ourAddress() const { return our_address; } + +private: + LoggerPtr log; + bool chunked = false; + UInt32 * last_finish_chunk = nullptr; // pointer to the last chunk header created by finishChunk + bool chunk_started = false; // chunk started flag + UInt32 * chunk_size_ptr = nullptr; // pointer to the chunk size holder in the buffer + size_t finishing = sizeof(*chunk_size_ptr); // indicates not enough buffer for end-of-chunk marker +}; + +} diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 59c98491c148..dd9e35834ebb 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -113,6 +113,9 @@ Cluster::Address::Address( secure = ConfigHelper::getBool(config, config_prefix + ".secure", false, /* empty_as */true) ? Protocol::Secure::Enable : Protocol::Secure::Disable; priority = Priority{config.getInt(config_prefix + ".priority", 1)}; + proto_send_chunked = config.getString(config_prefix + ".proto_caps.send", "notchunked"); + proto_recv_chunked = config.getString(config_prefix + ".proto_caps.recv", "notchunked"); + const char * port_type = secure == Protocol::Secure::Enable ? "tcp_port_secure" : "tcp_port"; auto default_port = config.getInt(port_type, 0); @@ -425,7 +428,9 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, auto pool = ConnectionPoolFactory::instance().get( static_cast(settings.distributed_connections_pool_size), address.host_name, address.port, - address.default_database, address.user, address.password, address.quota_key, + address.default_database, address.user, address.password, + address.proto_send_chunked, address.proto_recv_chunked, + address.quota_key, address.cluster, address.cluster_secret, "server", address.compression, address.secure, address.priority); @@ -589,6 +594,8 @@ void Cluster::addShard( replica.default_database, replica.user, replica.password, + replica.proto_send_chunked, + replica.proto_recv_chunked, replica.quota_key, replica.cluster, replica.cluster_secret, @@ -744,6 +751,8 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti address.default_database, address.user, address.password, + address.proto_send_chunked, + address.proto_recv_chunked, address.quota_key, address.cluster, address.cluster_secret, diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index dc5790ac339b..c69d77668abf 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -114,6 +114,8 @@ class Cluster UInt16 port{0}; String user; String password; + String proto_send_chunked = "notchunked"; + String proto_recv_chunked = "notchunked"; String quota_key; /// For inter-server authorization diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index d04a73e384e2..3d1181fe9da4 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include namespace DB @@ -38,6 +39,7 @@ namespace ErrorCodes extern const int TOO_LARGE_DISTRIBUTED_DEPTH; extern const int LOGICAL_ERROR; extern const int UNEXPECTED_CLUSTER; + extern const int INCONSISTENT_CLUSTER_DEFINITION; } namespace ClusterProxy @@ -240,7 +242,7 @@ getShardFilterGeneratorForCustomKey(const Cluster & cluster, ContextPtr context, return [my_custom_key_ast = std::move(custom_key_ast), column_description = columns, - custom_key_type = settings.parallel_replicas_custom_key_filter_type.value, + custom_key_type = settings.parallel_replicas_mode.value, custom_key_range_lower = settings.parallel_replicas_custom_key_range_lower.value, custom_key_range_upper = settings.parallel_replicas_custom_key_range_upper.value, query_context = context, @@ -286,7 +288,7 @@ void executeQuery( auto cluster = query_info.getCluster(); auto new_context = updateSettingsAndClientInfoForCluster(*cluster, is_remote_function, context, settings, main_table, query_info.additional_filter_ast, log, &distributed_settings); - if (context->getSettingsRef().allow_experimental_parallel_reading_from_replicas + if (context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value && context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value != new_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value) { @@ -439,7 +441,8 @@ void executeQueryWithParallelReplicas( QueryProcessingStage::Enum processed_stage, const ASTPtr & query_ast, ContextPtr context, - std::shared_ptr storage_limits) + std::shared_ptr storage_limits, + QueryPlanStepPtr analyzed_read_from_merge_tree) { auto logger = getLogger("executeQueryWithParallelReplicas"); LOG_DEBUG(logger, "Executing read from {}, header {}, query ({}), stage {} with parallel replicas", @@ -459,14 +462,14 @@ void executeQueryWithParallelReplicas( { LOG_WARNING( getLogger("executeQueryWithParallelReplicas"), - "Setting 'use_hedged_requests' explicitly with enabled 'allow_experimental_parallel_reading_from_replicas' has no effect. " + "Setting 'use_hedged_requests' explicitly with enabled 'enable_parallel_replicas' has no effect. " "Hedged connections are not used for parallel reading from replicas"); } else { LOG_INFO( getLogger("executeQueryWithParallelReplicas"), - "Disabling 'use_hedged_requests' in favor of 'allow_experimental_parallel_reading_from_replicas'. Hedged connections are " + "Disabling 'use_hedged_requests' in favor of 'enable_parallel_replicas'. Hedged connections are " "not used for parallel reading from replicas"); } @@ -485,12 +488,12 @@ void executeQueryWithParallelReplicas( shard_num = column->getUInt(0); } - const auto shard_count = not_optimized_cluster->getShardCount(); ClusterPtr new_cluster = not_optimized_cluster; /// if got valid shard_num from query initiator, then parallel replicas scope is the specified shard /// shards are numbered in order of appearance in the cluster config if (shard_num > 0) { + const auto shard_count = not_optimized_cluster->getShardCount(); if (shard_num > shard_count) throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -516,21 +519,147 @@ void executeQueryWithParallelReplicas( "`cluster_for_parallel_replicas` setting refers to cluster with several shards. Expected a cluster with one shard"); } + const auto & shard = new_cluster->getShardsInfo().at(0); + size_t max_replicas_to_use = settings.max_parallel_replicas; + if (max_replicas_to_use > shard.getAllNodeCount()) + { + LOG_INFO( + getLogger("ReadFromParallelRemoteReplicasStep"), + "The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). " + "Will use the latter number to execute the query.", + settings.max_parallel_replicas, + shard.getAllNodeCount()); + max_replicas_to_use = shard.getAllNodeCount(); + } + + auto coordinator = std::make_shared(max_replicas_to_use); + auto external_tables = new_context->getExternalTables(); - auto read_from_remote = std::make_unique( - query_ast, - new_cluster, - storage_id, - header, - processed_stage, - new_context, - getThrottler(new_context), - std::move(scalars), - std::move(external_tables), - getLogger("ReadFromParallelRemoteReplicasStep"), - std::move(storage_limits)); - - query_plan.addStep(std::move(read_from_remote)); + + std::vector shuffled_pool; + if (max_replicas_to_use < shard.getAllNodeCount()) + { + // will be shuffled according to `load_balancing` setting + shuffled_pool = shard.pool->getShuffledPools(settings); + } + else + { + /// If all replicas in cluster are used for query execution, + /// try to preserve replicas order as in cluster definition. + /// It's important for data locality during query execution + /// independently of the query initiator + auto priority_func = [](size_t i) { return Priority{static_cast(i)}; }; + shuffled_pool = shard.pool->getShuffledPools(settings, priority_func); + } + + std::vector pools_to_use; + pools_to_use.reserve(shuffled_pool.size()); + for (auto & pool : shuffled_pool) + pools_to_use.emplace_back(std::move(pool.pool)); + + /// do not build local plan for distributed queries for now (address it later) + if (settings.allow_experimental_analyzer && settings.parallel_replicas_local_plan && !shard_num) + { + /// find local replica index in pool + std::optional local_replica_index; + for (size_t i = 0, s = pools_to_use.size(); i < s; ++i) + { + const auto & hostname = pools_to_use[i]->getHost(); + const auto found = std::find_if( + begin(shard.local_addresses), + end(shard.local_addresses), + [&hostname](const Cluster::Address & local_addr) { return hostname == local_addr.host_name; }); + if (found != shard.local_addresses.end()) + { + local_replica_index = i; + break; + } + } + if (!local_replica_index) + throw Exception( + ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, + "Local replica is not found in '{}' cluster definition, see 'cluster_for_parallel_replicas' setting", + new_cluster->getName()); + + // resize the pool but keep local replicas in it (and update its index) + chassert(max_replicas_to_use <= pools_to_use.size()); + if (local_replica_index >= max_replicas_to_use) + { + std::swap(pools_to_use[max_replicas_to_use - 1], pools_to_use[local_replica_index.value()]); + local_replica_index = max_replicas_to_use - 1; + } + pools_to_use.resize(max_replicas_to_use); + + auto [local_plan, with_parallel_replicas] = createLocalPlanForParallelReplicas( + query_ast, + header, + new_context, + processed_stage, + coordinator, + std::move(analyzed_read_from_merge_tree), + local_replica_index.value()); + + if (!with_parallel_replicas) + { + query_plan = std::move(*local_plan); + return; + } + + LOG_DEBUG(logger, "Local replica got replica number {}", local_replica_index.value()); + + auto read_from_remote = std::make_unique( + query_ast, + new_cluster, + storage_id, + coordinator, + header, + processed_stage, + new_context, + getThrottler(new_context), + std::move(scalars), + std::move(external_tables), + getLogger("ReadFromParallelRemoteReplicasStep"), + std::move(storage_limits), + std::move(pools_to_use), + local_replica_index); + + auto remote_plan = std::make_unique(); + remote_plan->addStep(std::move(read_from_remote)); + + DataStreams input_streams; + input_streams.reserve(2); + input_streams.emplace_back(local_plan->getCurrentDataStream()); + input_streams.emplace_back(remote_plan->getCurrentDataStream()); + + std::vector plans; + plans.emplace_back(std::move(local_plan)); + plans.emplace_back(std::move(remote_plan)); + + auto union_step = std::make_unique(std::move(input_streams)); + query_plan.unitePlans(std::move(union_step), std::move(plans)); + } + else + { + chassert(max_replicas_to_use <= pools_to_use.size()); + pools_to_use.resize(max_replicas_to_use); + + auto read_from_remote = std::make_unique( + query_ast, + new_cluster, + storage_id, + std::move(coordinator), + header, + processed_stage, + new_context, + getThrottler(new_context), + std::move(scalars), + std::move(external_tables), + getLogger("ReadFromParallelRemoteReplicasStep"), + std::move(storage_limits), + std::move(pools_to_use)); + + query_plan.addStep(std::move(read_from_remote)); + } } void executeQueryWithParallelReplicas( @@ -540,7 +669,8 @@ void executeQueryWithParallelReplicas( const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context, ContextPtr context, - std::shared_ptr storage_limits) + std::shared_ptr storage_limits, + QueryPlanStepPtr analyzed_read_from_merge_tree) { QueryTreeNodePtr modified_query_tree = query_tree->clone(); rewriteJoinToGlobalJoin(modified_query_tree, context); @@ -550,7 +680,8 @@ void executeQueryWithParallelReplicas( = InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_tree, context, SelectQueryOptions(processed_stage).analyze()); auto modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree); - executeQueryWithParallelReplicas(query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits); + executeQueryWithParallelReplicas( + query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits, std::move(analyzed_read_from_merge_tree)); } void executeQueryWithParallelReplicas( @@ -646,6 +777,49 @@ void executeQueryWithParallelReplicasCustomKey( context, query_info.query, storage_id.getDatabaseName(), storage_id.getTableName(), /*table_function_ptr=*/nullptr); executeQueryWithParallelReplicasCustomKey(query_plan, storage_id, query_info, columns, snapshot, processed_stage, header, context); } + +bool canUseParallelReplicasOnInitiator(const ContextPtr & context) +{ + if (!context->canUseParallelReplicasOnInitiator()) + return false; + + auto cluster = context->getClusterForParallelReplicas(); + if (cluster->getShardCount() == 1) + return cluster->getShardsInfo()[0].getAllNodeCount() > 1; + + /// parallel replicas with distributed table + auto scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; + UInt64 shard_num = 0; /// shard_num is 1-based, so 0 - no shard specified + const auto it = scalars.find("_shard_num"); + if (it != scalars.end()) + { + const Block & block = it->second; + const auto & column = block.safeGetByPosition(0).column; + shard_num = column->getUInt(0); + } + if (shard_num > 0) + { + const auto shard_count = cluster->getShardCount(); + if (shard_num > shard_count) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Shard number is greater than shard count: shard_num={} shard_count={} cluster={}", + shard_num, + shard_count, + cluster->getName()); + + return cluster->getShardsInfo().at(shard_num - 1).getAllNodeCount() > 1; + } + + if (cluster->getShardCount() > 1) + throw DB::Exception( + ErrorCodes::UNEXPECTED_CLUSTER, + "`cluster_for_parallel_replicas` setting refers to cluster with {} shards. Expected a cluster with one shard", + cluster->getShardCount()); + + return false; +} + } } diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index c22fcd24f03c..2a21f3e82551 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -35,6 +35,9 @@ using QueryTreeNodePtr = std::shared_ptr; class PlannerContext; using PlannerContextPtr = std::shared_ptr; +class IQueryPlanStep; +using QueryPlanStepPtr = std::unique_ptr; + namespace ClusterProxy { @@ -55,6 +58,8 @@ using AdditionalShardFilterGenerator = std::function; AdditionalShardFilterGenerator getShardFilterGeneratorForCustomKey(const Cluster & cluster, ContextPtr context, const ColumnsDescription & columns); +bool canUseParallelReplicasOnInitiator(const ContextPtr & context); + /// Execute a distributed query, creating a query plan, from which the query pipeline can be built. /// `stream_factory` object encapsulates the logic of creating plans for a different type of query /// (currently SELECT, DESCRIBE). @@ -81,7 +86,8 @@ void executeQueryWithParallelReplicas( QueryProcessingStage::Enum processed_stage, const ASTPtr & query_ast, ContextPtr context, - std::shared_ptr storage_limits); + std::shared_ptr storage_limits, + QueryPlanStepPtr read_from_merge_tree = nullptr); void executeQueryWithParallelReplicas( QueryPlan & query_plan, @@ -98,7 +104,8 @@ void executeQueryWithParallelReplicas( const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context, ContextPtr context, - std::shared_ptr storage_limits); + std::shared_ptr storage_limits, + QueryPlanStepPtr read_from_merge_tree); void executeQueryWithParallelReplicasCustomKey( QueryPlan & query_plan, diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a967503dfb56..d12ff92b0f1b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1367,7 +1367,7 @@ ConfigurationPtr Context::getUsersConfig() return shared->users_config; } -void Context::setUser(const UUID & user_id_) +void Context::setUser(const UUID & user_id_, const std::vector & external_roles_) { /// Prepare lists of user's profiles, constraints, settings, roles. /// NOTE: AccessControl::read() and other AccessControl's functions may require some IO work, @@ -1382,7 +1382,6 @@ void Context::setUser(const UUID & user_id_) const auto & database = user->default_database; /// Apply user's profiles, constraints, settings, roles. - std::lock_guard lock(mutex); setUserIDWithLock(user_id_, lock); @@ -1392,6 +1391,7 @@ void Context::setUser(const UUID & user_id_) setCurrentProfilesWithLock(*enabled_profiles, /* check_constraints= */ false, lock); setCurrentRolesWithLock(default_roles, lock); + setExternalRolesWithLock(external_roles_, lock); /// It's optional to specify the DEFAULT DATABASE in the user's definition. if (!database.empty()) @@ -1435,6 +1435,18 @@ void Context::setCurrentRolesWithLock(const std::vector & new_current_role need_recalculate_access = true; } +void Context::setExternalRolesWithLock(const std::vector & new_external_roles, const std::lock_guard &) +{ + if (!new_external_roles.empty()) + { + if (current_roles) + current_roles->insert(current_roles->end(), new_external_roles.begin(), new_external_roles.end()); + else + current_roles = std::make_shared>(new_external_roles); + need_recalculate_access = true; + } +} + void Context::setCurrentRolesImpl(const std::vector & new_current_roles, bool throw_if_not_granted, bool skip_if_not_granted, const std::shared_ptr & user) { if (skip_if_not_granted) @@ -1548,7 +1560,8 @@ std::shared_ptr Context::getAccess() const /// If setUserID() was never called then this must be the global context with the full access. bool full_access = !user_id; - return ContextAccessParams{user_id, full_access, /* use_default_roles= */ false, current_roles, *settings, current_database, client_info}; + return ContextAccessParams{ + user_id, full_access, /* use_default_roles= */ false, current_roles, external_roles, *settings, current_database, client_info}; }; /// Check if the current access rights are still valid, otherwise get parameters for recalculating access rights. @@ -5594,24 +5607,13 @@ std::shared_ptr Context::getAsyncReadCounters() const return async_read_counters; } -Context::ParallelReplicasMode Context::getParallelReplicasMode() const -{ - const auto & settings_ref = getSettingsRef(); - - using enum Context::ParallelReplicasMode; - if (!settings_ref.parallel_replicas_custom_key.value.empty()) - return CUSTOM_KEY; - - if (settings_ref.allow_experimental_parallel_reading_from_replicas > 0) - return READ_TASKS; - - return SAMPLE_KEY; -} - bool Context::canUseTaskBasedParallelReplicas() const { const auto & settings_ref = getSettingsRef(); - return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS && settings_ref.max_parallel_replicas > 1; + + return settings_ref.allow_experimental_parallel_reading_from_replicas > 0 + && settings_ref.parallel_replicas_mode == ParallelReplicasMode::READ_TASKS + && settings_ref.max_parallel_replicas > 1; } bool Context::canUseParallelReplicasOnInitiator() const @@ -5626,7 +5628,15 @@ bool Context::canUseParallelReplicasOnFollower() const bool Context::canUseParallelReplicasCustomKey() const { - return settings->max_parallel_replicas > 1 && getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY; + const auto & settings_ref = getSettingsRef(); + + const bool has_enough_servers = settings_ref.max_parallel_replicas > 1; + const bool parallel_replicas_enabled = settings_ref.allow_experimental_parallel_reading_from_replicas > 0; + const bool is_parallel_replicas_with_custom_key = + settings_ref.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY_SAMPLING || + settings_ref.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY_RANGE; + + return has_enough_servers && parallel_replicas_enabled && is_parallel_replicas_with_custom_key; } bool Context::canUseParallelReplicasCustomKeyForCluster(const Cluster & cluster) const @@ -5636,8 +5646,23 @@ bool Context::canUseParallelReplicasCustomKeyForCluster(const Cluster & cluster) bool Context::canUseOffsetParallelReplicas() const { - return offset_parallel_replicas_enabled && settings->max_parallel_replicas > 1 - && getParallelReplicasMode() != Context::ParallelReplicasMode::READ_TASKS; + const auto & settings_ref = getSettingsRef(); + + /** + * Offset parallel replicas algorithm is not only the one which relies on native SAMPLING KEY, + * but also those which rely on customer-provided "custom" key. + * We combine them together into one group for convenience. + */ + const bool has_enough_servers = settings_ref.max_parallel_replicas > 1; + const bool parallel_replicas_enabled = settings_ref.allow_experimental_parallel_reading_from_replicas > 0; + const bool is_parallel_replicas_with_custom_key_or_native_sampling_key = + settings_ref.parallel_replicas_mode == ParallelReplicasMode::SAMPLING_KEY || + settings_ref.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY_SAMPLING || + settings_ref.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY_RANGE; + return offset_parallel_replicas_enabled && + has_enough_servers && + parallel_replicas_enabled && + is_parallel_replicas_with_custom_key_or_native_sampling_key; } void Context::disableOffsetParallelReplicas() diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 4b52d28dd9ed..a4fa924d89b8 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -277,6 +277,7 @@ class ContextData std::optional user_id; std::shared_ptr> current_roles; + std::shared_ptr> external_roles; std::shared_ptr settings_constraints_and_current_profiles; mutable std::shared_ptr access; mutable bool need_recalculate_access = true; @@ -613,7 +614,7 @@ class Context: public ContextData, public std::enable_shared_from_this /// Sets the current user assuming that he/she is already authenticated. /// WARNING: This function doesn't check password! - void setUser(const UUID & user_id_); + void setUser(const UUID & user_id_, const std::vector & external_roles_ = {}); UserPtr getUser() const; std::optional getUserID() const; @@ -1332,15 +1333,6 @@ class Context: public ContextData, public std::enable_shared_from_this ClusterPtr getClusterForParallelReplicas() const; - enum class ParallelReplicasMode : uint8_t - { - SAMPLE_KEY, - CUSTOM_KEY, - READ_TASKS, - }; - - ParallelReplicasMode getParallelReplicasMode() const; - void setPreparedSetsCache(const PreparedSetsCachePtr & cache); PreparedSetsCachePtr getPreparedSetsCache() const; @@ -1357,6 +1349,8 @@ class Context: public ContextData, public std::enable_shared_from_this void setCurrentRolesWithLock(const std::vector & new_current_roles, const std::lock_guard & lock); + void setExternalRolesWithLock(const std::vector & new_external_roles, const std::lock_guard & lock); + void setSettingWithLock(std::string_view name, const String & value, const std::lock_guard & lock); void setSettingWithLock(std::string_view name, const Field & value, const std::lock_guard & lock); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d3334548bc63..9741519801b7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -597,7 +597,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( settings.parallel_replicas_count, settings.parallel_replica_offset, std::move(custom_key_ast), - {settings.parallel_replicas_custom_key_filter_type, + {settings.parallel_replicas_mode, settings.parallel_replicas_custom_key_range_lower, settings.parallel_replicas_custom_key_range_upper}, storage->getInMemoryMetadataPtr()->columns, @@ -2314,12 +2314,12 @@ void InterpreterSelectQuery::addPrewhereAliasActions() } /// Based on the query analysis, check if using a trivial count (storage or partition metadata) is possible -std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_parallel_replicas) +std::optional InterpreterSelectQuery::getTrivialCount(UInt64 allow_experimental_parallel_reading_from_replicas) { const Settings & settings = context->getSettingsRef(); bool optimize_trivial_count = syntax_analyzer_result->optimize_trivial_count - && (max_parallel_replicas <= 1) + && (allow_experimental_parallel_reading_from_replicas == 0) && !settings.allow_experimental_query_deduplication && !settings.empty_result_for_aggregation_by_empty_set && storage @@ -2403,7 +2403,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc std::optional num_rows; /// Optimization for trivial query like SELECT count() FROM table. - if (processing_stage == QueryProcessingStage::FetchColumns && (num_rows = getTrivialCount(settings.max_parallel_replicas))) + if (processing_stage == QueryProcessingStage::FetchColumns && (num_rows = getTrivialCount(settings.allow_experimental_parallel_reading_from_replicas))) { const auto & desc = query_analyzer->aggregates()[0]; const auto & func = desc.function; diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index a01a390f3c76..d70d9ea3eea0 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -195,7 +195,7 @@ class InterpreterSelectQuery : public IInterpreterUnionOrSelectQuery void executeExtremes(QueryPlan & query_plan); void executeSubqueriesInSetsAndJoins(QueryPlan & query_plan); bool autoFinalOnQuery(ASTSelectQuery & select_query); - std::optional getTrivialCount(UInt64 max_parallel_replicas); + std::optional getTrivialCount(UInt64 allow_experimental_parallel_reading_from_replicas); /// Check if we can limit block size to read based on LIMIT clause UInt64 maxBlockSizeByLimit() const; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 58e9c4695686..61a432cde292 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -147,6 +147,7 @@ ColumnDependencies getAllColumnDependencies( bool isStorageTouchedByMutations( MergeTreeData::DataPartPtr source_part, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, ContextPtr context) @@ -154,7 +155,7 @@ bool isStorageTouchedByMutations( if (commands.empty()) return false; - auto storage_from_part = std::make_shared(source_part); + auto storage_from_part = std::make_shared(source_part, mutations_snapshot); bool all_commands_can_be_skipped = true; for (const auto & command : commands) @@ -285,8 +286,13 @@ MutationsInterpreter::Source::Source(StoragePtr storage_) : storage(std::move(st { } -MutationsInterpreter::Source::Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_) - : data(&storage_), part(std::move(source_part_)) +MutationsInterpreter::Source::Source( + MergeTreeData & storage_, + MergeTreeData::DataPartPtr source_part_, + AlterConversionsPtr alter_conversions_) + : data(&storage_) + , part(std::move(source_part_)) + , alter_conversions(std::move(alter_conversions_)) { } @@ -386,13 +392,14 @@ MutationsInterpreter::MutationsInterpreter( MutationsInterpreter::MutationsInterpreter( MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, + AlterConversionsPtr alter_conversions_, StorageMetadataPtr metadata_snapshot_, MutationCommands commands_, Names available_columns_, ContextPtr context_, Settings settings_) : MutationsInterpreter( - Source(storage_, std::move(source_part_)), + Source(storage_, std::move(source_part_), std::move(alter_conversions_)), std::move(metadata_snapshot_), std::move(commands_), std::move(available_columns_), std::move(context_), std::move(settings_)) { @@ -1212,7 +1219,7 @@ void MutationsInterpreter::Source::read( createReadFromPartStep( MergeTreeSequentialSourceType::Mutation, plan, *data, storage_snapshot, - part, required_columns, + part, alter_conversions, required_columns, apply_deleted_mask_, std::move(filter), context_, getLogger("MutationsInterpreter")); } diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 57863e9ae737..8601558b7882 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -20,6 +20,7 @@ using QueryPipelineBuilderPtr = std::unique_ptr; /// Return false if the data isn't going to be changed by mutations. bool isStorageTouchedByMutations( MergeTreeData::DataPartPtr source_part, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, ContextPtr context @@ -70,6 +71,7 @@ class MutationsInterpreter MutationsInterpreter( MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, + AlterConversionsPtr alter_conversions_, StorageMetadataPtr metadata_snapshot_, MutationCommands commands_, Names available_columns_, @@ -137,7 +139,7 @@ class MutationsInterpreter bool can_execute_) const; explicit Source(StoragePtr storage_); - Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_); + Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, AlterConversionsPtr alter_conversions_); private: StoragePtr storage; @@ -145,6 +147,7 @@ class MutationsInterpreter /// Special case for *MergeTree. MergeTreeData * data = nullptr; MergeTreeData::DataPartPtr part; + AlterConversionsPtr alter_conversions; }; private: diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 9b74ae2aa930..e7e3d1f84b28 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include #include @@ -25,7 +27,6 @@ #include #include - namespace DB { @@ -286,7 +287,7 @@ void Session::shutdownNamedSessions() Session::Session(const ContextPtr & global_context_, ClientInfo::Interface interface_, bool is_secure, const std::string & certificate) : auth_id(UUIDHelpers::generateV4()), global_context(global_context_), - log(getLogger(String{magic_enum::enum_name(interface_)} + "-Session")) + log(getLogger(String{magic_enum::enum_name(interface_)} + "-Session-" + toString(auth_id))) { prepared_client_info.emplace(); prepared_client_info->interface = interface_; @@ -331,12 +332,12 @@ AuthenticationType Session::getAuthenticationTypeOrLogInFailure(const String & u } } -void Session::authenticate(const String & user_name, const String & password, const Poco::Net::SocketAddress & address) +void Session::authenticate(const String & user_name, const String & password, const Poco::Net::SocketAddress & address, const Strings & external_roles_) { - authenticate(BasicCredentials{user_name, password}, address); + authenticate(BasicCredentials{user_name, password}, address, external_roles_); } -void Session::authenticate(const Credentials & credentials_, const Poco::Net::SocketAddress & address_) +void Session::authenticate(const Credentials & credentials_, const Poco::Net::SocketAddress & address_, const Strings & external_roles_) { if (session_context) throw Exception(ErrorCodes::LOGICAL_ERROR, "If there is a session context it must be created after authentication"); @@ -348,8 +349,8 @@ void Session::authenticate(const Credentials & credentials_, const Poco::Net::So if ((address == Poco::Net::SocketAddress{}) && (prepared_client_info->interface == ClientInfo::Interface::LOCAL)) address = Poco::Net::SocketAddress{"127.0.0.1", 0}; - LOG_DEBUG(log, "{} Authenticating user '{}' from {}", - toString(auth_id), credentials_.getUserName(), address.toString()); + LOG_DEBUG(log, "Authenticating user '{}' from {}", + credentials_.getUserName(), address.toString()); try { @@ -358,6 +359,14 @@ void Session::authenticate(const Credentials & credentials_, const Poco::Net::So settings_from_auth_server = auth_result.settings; LOG_DEBUG(log, "{} Authenticated with global context as user {}", toString(auth_id), toString(*user_id)); + + if (!external_roles_.empty() && global_context->getSettingsRef().push_external_roles_in_interserver_queries) + { + external_roles = global_context->getAccessControl().find(external_roles_); + + LOG_DEBUG(log, "User {} has external_roles applied: [{}] ({})", + toString(*user_id), fmt::join(external_roles_, ", "), external_roles_.size()); + } } catch (const Exception & e) { @@ -382,7 +391,7 @@ void Session::checkIfUserIsStillValid() void Session::onAuthenticationFailure(const std::optional & user_name, const Poco::Net::SocketAddress & address_, const Exception & e) { - LOG_DEBUG(log, "{} Authentication failed with error: {}", toString(auth_id), e.what()); + LOG_DEBUG(log, "Authentication failed with error: {}", e.what()); if (auto session_log = getSessionLog()) { /// Add source address to the log @@ -508,8 +517,8 @@ ContextMutablePtr Session::makeSessionContext() if (session_tracker_handle) throw Exception(ErrorCodes::LOGICAL_ERROR, "Session tracker handle was created before making session"); - LOG_DEBUG(log, "{} Creating session context with user_id: {}", - toString(auth_id), toString(*user_id)); + LOG_DEBUG(log, "Creating session context with user_id: {}", + toString(*user_id)); /// Make a new session context. ContextMutablePtr new_session_context; new_session_context = Context::createCopy(global_context); @@ -520,7 +529,7 @@ ContextMutablePtr Session::makeSessionContext() prepared_client_info.reset(); /// Set user information for the new context: current profiles, roles, access rights. - new_session_context->setUser(*user_id); + new_session_context->setUser(*user_id, external_roles); /// Session context is ready. session_context = new_session_context; @@ -551,8 +560,8 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: if (session_tracker_handle) throw Exception(ErrorCodes::LOGICAL_ERROR, "Session tracker handle was created before making session"); - LOG_DEBUG(log, "{} Creating named session context with name: {}, user_id: {}", - toString(auth_id), session_name_, toString(*user_id)); + LOG_DEBUG(log, "Creating named session context with name: {}, user_id: {}", + session_name_, toString(*user_id)); /// Make a new session context OR /// if the `session_id` and `user_id` were used before then just get a previously created session context. @@ -575,7 +584,7 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: /// Set user information for the new context: current profiles, roles, access rights. if (!access->tryGetUser()) { - new_session_context->setUser(*user_id); + new_session_context->setUser(*user_id, external_roles); max_sessions_for_user = new_session_context->getSettingsRef().max_sessions_for_user; } else @@ -627,7 +636,7 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t throw Exception(ErrorCodes::LOGICAL_ERROR, "Query context must be created after authentication"); /// We can create a query context either from a session context or from a global context. - bool from_session_context = static_cast(session_context); + const bool from_session_context = static_cast(session_context); /// Create a new query context. ContextMutablePtr query_context = Context::createCopy(from_session_context ? session_context : global_context); @@ -667,7 +676,7 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t /// Set user information for the new context: current profiles, roles, access rights. if (user_id && !query_context->getAccess()->tryGetUser()) - query_context->setUser(*user_id); + query_context->setUser(*user_id, external_roles); /// Query context is ready. query_context_created = true; diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index fc41c78e666d..800aadb001e1 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace Poco::Net { class SocketAddress; } @@ -50,8 +51,11 @@ class Session /// Sets the current user, checks the credentials and that the specified address is allowed to connect from. /// The function throws an exception if there is no such user or password is wrong. - void authenticate(const String & user_name, const String & password, const Poco::Net::SocketAddress & address); - void authenticate(const Credentials & credentials_, const Poco::Net::SocketAddress & address_); + void authenticate(const String & user_name, const String & password, const Poco::Net::SocketAddress & address, const Strings & external_roles_ = {}); + + /// `external_roles_` names of the additional roles (over what is granted via local access control mechanisms) that would be granted to user during this session. + /// Role is not granted if it can't be found by name via AccessControl (i.e. doesn't exist on this instance). + void authenticate(const Credentials & credentials_, const Poco::Net::SocketAddress & address_, const Strings & external_roles_ = {}); // Verifies whether the user's validity extends beyond the current time. // Throws an exception if the user's validity has expired. @@ -114,6 +118,8 @@ class Session mutable UserPtr user; std::optional user_id; + std::vector external_roles; + ContextMutablePtr session_context; mutable bool query_context_created = false; diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 6f902c7cd7e9..7b17cecefb08 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -31,7 +31,8 @@ ASTPtr getCustomKeyFilterForParallelReplica( const ContextPtr & context) { chassert(replicas_count > 1); - if (filter.filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT) + chassert(filter.filter_type == ParallelReplicasMode::CUSTOM_KEY_SAMPLING || filter.filter_type == ParallelReplicasMode::CUSTOM_KEY_RANGE); + if (filter.filter_type == ParallelReplicasMode::CUSTOM_KEY_SAMPLING) { // first we do modulo with replica count auto modulo_function = makeASTFunction("positiveModulo", custom_key_ast, std::make_shared(replicas_count)); @@ -42,7 +43,7 @@ ASTPtr getCustomKeyFilterForParallelReplica( return equals_function; } - chassert(filter.filter_type == ParallelReplicasCustomKeyFilterType::RANGE); + chassert(filter.filter_type == ParallelReplicasMode::CUSTOM_KEY_RANGE); KeyDescription custom_key_description = KeyDescription::getKeyFromAST(custom_key_ast, columns, context); diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h index dfee5123ecb1..28fa1e45ebbf 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h @@ -12,7 +12,7 @@ namespace DB { struct ParallelReplicasCustomKeyFilter { - ParallelReplicasCustomKeyFilterType filter_type; + ParallelReplicasMode filter_type; UInt64 range_lower; UInt64 range_upper; }; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index bc31af32a20c..1754b2102f63 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -3,6 +3,7 @@ #include #include +#include #include @@ -290,14 +291,21 @@ bool applyTrivialCountIfPossible( if (!num_rows) return false; - if (settings.max_parallel_replicas > 1) + if (settings.allow_experimental_parallel_reading_from_replicas > 0 && settings.max_parallel_replicas > 1) { - if (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas == 0) + /// Imagine the situation when we have a query with parallel replicas and + /// this code executed on the remote server. + /// If we will apply trivial count optimization, then each remote server will do the same + /// and we will have N times more rows as the result on the initiator. + /// TODO: This condition seems unneeded when we will make the parallel replicas with custom key + /// to work on top of MergeTree instead of Distributed. + if (settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY_RANGE || + settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY_SAMPLING || + settings.parallel_replicas_mode == ParallelReplicasMode::SAMPLING_KEY) return false; /// The query could use trivial count if it didn't use parallel replicas, so let's disable it query_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); - query_context->setSetting("max_parallel_replicas", UInt64{1}); LOG_TRACE(getLogger("Planner"), "Disabling parallel replicas to be able to use a trivial count optimization"); } @@ -504,7 +512,7 @@ std::optional buildCustomKeyFilterIfNeeded(const StoragePtr & sto settings.parallel_replicas_count, settings.parallel_replica_offset, std::move(custom_key_ast), - {settings.parallel_replicas_custom_key_filter_type, + {settings.parallel_replicas_mode, settings.parallel_replicas_custom_key_range_lower, settings.parallel_replicas_custom_key_range_upper}, storage->getInMemoryMetadataPtr()->columns, @@ -893,31 +901,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres /// query_plan can be empty if there is nothing to read if (query_plan.isInitialized() && parallel_replicas_enabled_for_storage(storage, settings)) { - // (1) find read step - QueryPlan::Node * node = query_plan.getRootNode(); - ReadFromMergeTree * reading = nullptr; - while (node) - { - reading = typeid_cast(node->step.get()); - if (reading) - break; - - QueryPlan::Node * prev_node = node; - if (!node->children.empty()) - { - chassert(node->children.size() == 1); - node = node->children.at(0); - } - else - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Step is expected to be ReadFromMergeTree but it's {}", - prev_node->step->getName()); - } - } - - chassert(reading); if (query_context->canUseParallelReplicasCustomKey() && query_context->getClientInfo().distributed_depth == 0) { if (auto cluster = query_context->getClusterForParallelReplicas(); @@ -940,21 +923,50 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres query_plan = std::move(query_plan_parallel_replicas); } } - else if (query_context->canUseParallelReplicasOnInitiator()) + else if (ClusterProxy::canUseParallelReplicasOnInitiator(query_context)) { + // (1) find read step + QueryPlan::Node * node = query_plan.getRootNode(); + ReadFromMergeTree * reading = nullptr; + while (node) + { + reading = typeid_cast(node->step.get()); + if (reading) + break; + + QueryPlan::Node * prev_node = node; + if (!node->children.empty()) + { + chassert(node->children.size() == 1); + node = node->children.at(0); + } + else + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Step is expected to be ReadFromMergeTree but it's {}", + prev_node->step->getName()); + } + } + + chassert(reading); + // (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read if (settings.parallel_replicas_min_number_of_rows_per_replica > 0) { auto result_ptr = reading->selectRangesToRead(); - UInt64 rows_to_read = result_ptr->selected_rows; + + reading->setAnalyzedResult(std::move(result_ptr)); + if (table_expression_query_info.trivial_limit > 0 && table_expression_query_info.trivial_limit < rows_to_read) rows_to_read = table_expression_query_info.trivial_limit; if (max_block_size_limited && (max_block_size_limited < rows_to_read)) rows_to_read = max_block_size_limited; - const size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; + const size_t number_of_replicas_to_use + = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; LOG_TRACE( getLogger("Planner"), "Estimated {} rows to read. It is enough work for {} parallel replicas", @@ -980,6 +992,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres { from_stage = QueryProcessingStage::WithMergeableState; QueryPlan query_plan_parallel_replicas; + QueryPlanStepPtr reading_step = std::move(node->step); ClusterProxy::executeQueryWithParallelReplicas( query_plan_parallel_replicas, storage->getStorageID(), @@ -987,9 +1000,24 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres table_expression_query_info.query_tree, table_expression_query_info.planner_context, query_context, - table_expression_query_info.storage_limits); + table_expression_query_info.storage_limits, + std::move(reading_step)); query_plan = std::move(query_plan_parallel_replicas); } + else + { + QueryPlan query_plan_no_parallel_replicas; + storage->read( + query_plan_no_parallel_replicas, + columns_names, + storage_snapshot, + table_expression_query_info, + query_context, + from_stage, + max_block_size, + max_streams); + query_plan = std::move(query_plan_no_parallel_replicas); + } } } diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index 39edb1e6516b..25481d06670a 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -52,7 +52,13 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre const auto & storage = table_node.getStorage(); /// Here we check StorageDummy as well, to support a query tree with replaced storages. if (std::dynamic_pointer_cast(storage) || typeid_cast(storage.get())) + { + /// parallel replicas is not supported with FINAL + if (table_node.getTableExpressionModifiers() && table_node.getTableExpressionModifiers()->hasFinal()) + return {}; + return res; + } return {}; } diff --git a/src/Processors/QueryPlan/ConvertingActions.cpp b/src/Processors/QueryPlan/ConvertingActions.cpp new file mode 100644 index 000000000000..b9703ef59cd5 --- /dev/null +++ b/src/Processors/QueryPlan/ConvertingActions.cpp @@ -0,0 +1,32 @@ +#include +#include +#include + +namespace DB +{ + +void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects) +{ + if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header)) + return; + + auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name; + + auto get_converting_dag = [mode](const Block & block_, const Block & header_) + { + /// Convert header structure to expected. + /// Also we ignore constants from result and replace it with constants from header. + /// It is needed for functions like `now64()` or `randConstant()` because their values may be different. + return ActionsDAG::makeConvertingActions( + block_.getColumnsWithTypeAndName(), + header_.getColumnsWithTypeAndName(), + mode, + true); + }; + + auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header); + auto converting = std::make_unique(plan.getCurrentDataStream(), std::move(convert_actions_dag)); + plan.addStep(std::move(converting)); +} + +} diff --git a/src/Processors/QueryPlan/ConvertingActions.h b/src/Processors/QueryPlan/ConvertingActions.h new file mode 100644 index 000000000000..6bdf9b8af9a6 --- /dev/null +++ b/src/Processors/QueryPlan/ConvertingActions.h @@ -0,0 +1,9 @@ +#pragma once + +namespace DB +{ +class QueryPlan; +class Block; + +void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects); +} diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index d8624a1c99b7..eb699858bdf5 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -2,43 +2,13 @@ #include #include -#include #include #include -#include +#include namespace DB { -namespace -{ - -void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects) -{ - if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header)) - return; - - auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name; - - auto get_converting_dag = [mode](const Block & block_, const Block & header_) - { - /// Convert header structure to expected. - /// Also we ignore constants from result and replace it with constants from header. - /// It is needed for functions like `now64()` or `randConstant()` because their values may be different. - return ActionsDAG::makeConvertingActions( - block_.getColumnsWithTypeAndName(), - header_.getColumnsWithTypeAndName(), - mode, - true); - }; - - auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header); - auto converting = std::make_unique(plan.getCurrentDataStream(), std::move(convert_actions_dag)); - plan.addStep(std::move(converting)); -} - -} - std::unique_ptr createLocalPlan( const ASTPtr & query_ast, const Block & header, diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 5df7d7b4e82d..2eac18960669 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -899,6 +900,18 @@ AggregationInputOrder buildInputOrderInfo(AggregatingStep & aggregating, QueryPl return {}; } +static bool readingFromParallelReplicas(const QueryPlan::Node * node) +{ + IQueryPlanStep * step = node->step.get(); + while (!node->children.empty()) + { + step = node->children.front()->step.get(); + node = node->children.front(); + } + + return typeid_cast(step); +} + void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) @@ -924,6 +937,16 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) std::vector infos; infos.reserve(node.children.size()); + for (const auto * child : union_node->children) + { + /// in case of parallel replicas + /// avoid applying read-in-order optimization for local replica + /// since it will lead to different parallel replicas modes + /// between local and remote nodes + if (readingFromParallelReplicas(child)) + return; + } + for (auto * child : union_node->children) { infos.push_back(buildInputOrderInfo(*sorting, *child, steps_to_update)); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index b31ee7ea53cf..35ac2c220ffb 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -757,7 +757,7 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu projection_reading = reader.readFromParts( /* parts = */ {}, - /* alter_conversions = */ {}, + reading->getMutationsSnapshot()->cloneEmpty(), best_candidate->dag.getRequiredColumnsNames(), proj_snapshot, projection_query_info, diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index b15f913fc195..654046cda899 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -199,7 +199,7 @@ std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod auto projection_reading = reader.readFromParts( /*parts=*/ {}, - /*alter_conversions=*/ {}, + reading->getMutationsSnapshot()->cloneEmpty(), required_columns, proj_snapshot, query_info_copy, diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 7414d479cc93..998b606ec57a 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -41,12 +41,19 @@ bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) if (reading->readsInOrder()) return false; + const auto & query_settings = reading->getContext()->getSettingsRef(); + // Currently projection don't support deduplication when moving parts between shards. - if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) + if (query_settings.allow_experimental_query_deduplication) return false; // Currently projection don't support settings which implicitly modify aggregate functions. - if (reading->getContext()->getSettingsRef().aggregate_functions_null_for_empty) + if (query_settings.aggregate_functions_null_for_empty) + return false; + + /// Don't use projections if have mutations to apply + /// because we need to apply them on original data. + if (query_settings.apply_mutations_on_fly && reading->getMutationsSnapshot()->hasDataMutations()) return false; return true; @@ -215,20 +222,15 @@ bool analyzeProjectionCandidate( { MergeTreeData::DataPartsVector projection_parts; MergeTreeData::DataPartsVector normal_parts; - std::vector alter_conversions; + for (const auto & part_with_ranges : parts_with_ranges) { const auto & created_projections = part_with_ranges.data_part->getProjectionParts(); auto it = created_projections.find(candidate.projection->name); if (it != created_projections.end() && !it->second->is_broken) - { projection_parts.push_back(it->second); - } else - { normal_parts.push_back(part_with_ranges.data_part); - alter_conversions.push_back(part_with_ranges.alter_conversions); - } } if (projection_parts.empty()) @@ -241,6 +243,7 @@ bool analyzeProjectionCandidate( auto projection_result_ptr = reader.estimateNumMarksToRead( std::move(projection_parts), + reading.getMutationsSnapshot()->cloneEmpty(), required_column_names, candidate.projection->metadata, projection_query_info, @@ -254,7 +257,7 @@ bool analyzeProjectionCandidate( if (!normal_parts.empty()) { /// TODO: We can reuse existing analysis_result by filtering out projection parts - auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts), std::move(alter_conversions)); + auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts)); if (normal_result_ptr->selected_marks != 0) { diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp new file mode 100644 index 000000000000..050044edd3ae --- /dev/null +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -0,0 +1,97 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +std::pair, bool> createLocalPlanForParallelReplicas( + const ASTPtr & query_ast, + const Block & header, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + ParallelReplicasReadingCoordinatorPtr coordinator, + QueryPlanStepPtr analyzed_read_from_merge_tree, + size_t replica_number) +{ + checkStackSize(); + + auto query_plan = std::make_unique(); + auto new_context = Context::createCopy(context); + + /// Do not push down limit to local plan, as it will break `rows_before_limit_at_least` counter. + if (processed_stage == QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit) + processed_stage = QueryProcessingStage::WithMergeableStateAfterAggregation; + + /// Do not apply AST optimizations, because query + /// is already optimized and some optimizations + /// can be applied only for non-distributed tables + /// and we can produce query, inconsistent with remote plans. + auto select_query_options = SelectQueryOptions(processed_stage).ignoreASTOptimizations(); + + /// For Analyzer, identifier in GROUP BY/ORDER BY/LIMIT BY lists has been resolved to + /// ConstantNode in QueryTree if it is an alias of a constant, so we should not replace + /// ConstantNode with ProjectionNode again(https://github.com/ClickHouse/ClickHouse/issues/62289). + new_context->setSetting("enable_positional_arguments", Field(false)); + new_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, new_context, select_query_options); + query_plan = std::make_unique(std::move(interpreter).extractQueryPlan()); + + QueryPlan::Node * node = query_plan->getRootNode(); + ReadFromMergeTree * reading = nullptr; + while (node) + { + reading = typeid_cast(node->step.get()); + if (reading) + break; + + if (!node->children.empty()) + node = node->children.at(0); + else + node = nullptr; + } + + if (!reading) + /// it can happened if merge tree table is empty, - it'll be replaced with ReadFromPreparedSource + return {std::move(query_plan), false}; + + ReadFromMergeTree::AnalysisResultPtr analyzed_result_ptr; + if (analyzed_read_from_merge_tree.get()) + { + auto * analyzed_merge_tree = typeid_cast(analyzed_read_from_merge_tree.get()); + if (analyzed_merge_tree) + analyzed_result_ptr = analyzed_merge_tree->getAnalyzedResult(); + } + + MergeTreeAllRangesCallback all_ranges_cb = [coordinator](InitialAllRangesAnnouncement announcement) + { coordinator->handleInitialAllRangesAnnouncement(std::move(announcement)); }; + + MergeTreeReadTaskCallback read_task_cb = [coordinator](ParallelReadRequest req) -> std::optional + { return coordinator->handleRequest(std::move(req)); }; + + auto read_from_merge_tree_parallel_replicas = reading->createLocalParallelReplicasReadingStep( + analyzed_result_ptr, std::move(all_ranges_cb), std::move(read_task_cb), replica_number); + node->step = std::move(read_from_merge_tree_parallel_replicas); + + addConvertingActions(*query_plan, header, /*has_missing_objects=*/false); + + return {std::move(query_plan), true}; +} + +} diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h new file mode 100644 index 000000000000..ab0bbeaeeff1 --- /dev/null +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h @@ -0,0 +1,19 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +std::pair, bool> createLocalPlanForParallelReplicas( + const ASTPtr & query_ast, + const Block & header, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + ParallelReplicasReadingCoordinatorPtr coordinator, + QueryPlanStepPtr read_from_merge_tree, + size_t replica_number); +} diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index 935af9148273..aaa1f53b5eeb 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -222,7 +222,6 @@ class RangesInDataPartsBuilder { ranges_in_data_parts.emplace_back( initial_ranges_in_data_parts[part_index].data_part, - initial_ranges_in_data_parts[part_index].alter_conversions, initial_ranges_in_data_parts[part_index].part_index_in_query, MarkRanges{mark_range}); part_index_to_initial_ranges_in_data_parts_index[it->second] = part_index; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 348019d7d107..142f4be119d9 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1,6 +1,8 @@ #include +#include #include +#include #include #include #include @@ -8,6 +10,8 @@ #include #include #include +#include +#include #include #include #include @@ -16,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -24,10 +29,11 @@ #include #include #include -#include #include -#include +#include +#include #include +#include #include #include #include @@ -41,11 +47,6 @@ #include #include #include -#include -#include -#include -#include -#include #include #include @@ -266,7 +267,7 @@ void ReadFromMergeTree::AnalysisResult::checkLimits(const Settings & settings, c ReadFromMergeTree::ReadFromMergeTree( MergeTreeData::DataPartsVector parts_, - std::vector alter_conversions_, + MergeTreeData::MutationsSnapshotPtr mutations_, Names all_column_names_, const MergeTreeData & data_, const SelectQueryInfo & query_info_, @@ -277,13 +278,16 @@ ReadFromMergeTree::ReadFromMergeTree( std::shared_ptr max_block_numbers_to_read_, LoggerPtr log_, AnalysisResultPtr analyzed_result_ptr_, - bool enable_parallel_reading) + bool enable_parallel_reading_, + std::optional all_ranges_callback_, + std::optional read_task_callback_, + std::optional number_of_current_replica_) : SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader( storage_snapshot_->getSampleBlockForColumns(all_column_names_), query_info_.prewhere_info)}, all_column_names_, query_info_, storage_snapshot_, context_) , reader_settings(getMergeTreeReaderSettings(context_, query_info_)) , prepared_parts(std::move(parts_)) - , alter_conversions_for_parts(std::move(alter_conversions_)) + , mutations_snapshot(std::move(mutations_)) , all_column_names(std::move(all_column_names_)) , data(data_) , actions_settings(ExpressionActionsSettings::fromContext(context_)) @@ -295,13 +299,21 @@ ReadFromMergeTree::ReadFromMergeTree( , max_block_numbers_to_read(std::move(max_block_numbers_to_read_)) , log(std::move(log_)) , analyzed_result_ptr(analyzed_result_ptr_) - , is_parallel_reading_from_replicas(enable_parallel_reading) + , is_parallel_reading_from_replicas(enable_parallel_reading_) , enable_remove_parts_from_snapshot_optimization(query_info_.merge_tree_enable_remove_parts_from_snapshot_optimization) + , number_of_current_replica(number_of_current_replica_) { if (is_parallel_reading_from_replicas) { - all_ranges_callback = context->getMergeTreeAllRangesCallback(); - read_task_callback = context->getMergeTreeReadTaskCallback(); + if (all_ranges_callback_.has_value()) + all_ranges_callback = all_ranges_callback_.value(); + else + all_ranges_callback = context->getMergeTreeAllRangesCallback(); + + if (read_task_callback_.has_value()) + read_task_callback = read_task_callback_.value(); + else + read_task_callback = context->getMergeTreeReadTaskCallback(); } const auto & settings = context->getSettingsRef(); @@ -335,20 +347,38 @@ ReadFromMergeTree::ReadFromMergeTree( enable_vertical_final); } +std::unique_ptr ReadFromMergeTree::createLocalParallelReplicasReadingStep( + AnalysisResultPtr analyzed_result_ptr_, + MergeTreeAllRangesCallback all_ranges_callback_, + MergeTreeReadTaskCallback read_task_callback_, + size_t replica_number) +{ + const bool enable_parallel_reading = true; + return std::make_unique( + prepared_parts, + mutations_snapshot, + all_column_names, + data, + getQueryInfo(), + getStorageSnapshot(), + getContext(), + block_size.max_block_size_rows, + requested_num_streams, + max_block_numbers_to_read, + log, + std::move(analyzed_result_ptr_), + enable_parallel_reading, + all_ranges_callback_, + read_task_callback_, + replica_number); +} -Pipe ReadFromMergeTree::readFromPoolParallelReplicas( - RangesInDataParts parts_with_range, - Names required_columns, - PoolSettings pool_settings) +Pipe ReadFromMergeTree::readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings) { const auto & client_info = context->getClientInfo(); - auto extension = ParallelReadingExtension - { - .all_callback = all_ranges_callback.value(), - .callback = read_task_callback.value(), - .number_of_current_replica = client_info.number_of_current_replica, - }; + auto extension = ParallelReadingExtension{ + all_ranges_callback.value(), read_task_callback.value(), number_of_current_replica.value_or(client_info.number_of_current_replica), context->getClusterForParallelReplicas()->getShardsInfo().at(0).getAllNodeCount()}; /// We have a special logic for local replica. It has to read less data, because in some cases it should /// merge states of aggregate functions or do some other important stuff other than reading from Disk. @@ -364,6 +394,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( auto pool = std::make_shared( std::move(extension), std::move(parts_with_range), + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -445,6 +476,7 @@ Pipe ReadFromMergeTree::readFromPool( { pool = std::make_shared( std::move(parts_with_range), + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -458,6 +490,7 @@ Pipe ReadFromMergeTree::readFromPool( { pool = std::make_shared( std::move(parts_with_range), + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -514,12 +547,10 @@ Pipe ReadFromMergeTree::readInOrder( if (is_parallel_reading_from_replicas) { const auto & client_info = context->getClientInfo(); - ParallelReadingExtension extension - { - .all_callback = all_ranges_callback.value(), - .callback = read_task_callback.value(), - .number_of_current_replica = client_info.number_of_current_replica, - }; + ParallelReadingExtension extension{ + all_ranges_callback.value(), + read_task_callback.value(), + number_of_current_replica.value_or(client_info.number_of_current_replica), context->getClusterForParallelReplicas()->getShardsInfo().at(0).getAllNodeCount()}; const auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; if (auto result = pool_settings.min_marks_for_concurrent_read * multiplier; canConvertTo(result)) @@ -538,6 +569,7 @@ Pipe ReadFromMergeTree::readInOrder( std::move(extension), mode, parts_with_ranges, + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -553,6 +585,7 @@ Pipe ReadFromMergeTree::readInOrder( has_limit_below_one_block, read_type, parts_with_ranges, + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -563,11 +596,12 @@ Pipe ReadFromMergeTree::readInOrder( context); } - /// Actually it means that parallel reading from replicas enabled - /// and we have to collaborate with initiator. - /// In this case we won't set approximate rows, because it will be accounted multiple times. - const auto in_order_limit = query_info.input_order_info ? query_info.input_order_info->limit : 0; - const bool set_total_rows_approx = !is_parallel_reading_from_replicas; + /// If parallel replicas enabled, set total rows in progress here only on initiator with local plan + /// Otherwise rows will counted multiple times + const UInt64 in_order_limit = query_info.input_order_info ? query_info.input_order_info->limit : 0; + const bool parallel_replicas_local_plan_for_initiator = is_parallel_reading_from_replicas + && context->getSettingsRef().parallel_replicas_local_plan && context->canUseParallelReplicasOnInitiator(); + const bool set_total_rows_approx = !is_parallel_reading_from_replicas || parallel_replicas_local_plan_for_initiator; Pipes pipes; for (size_t i = 0; i < parts_with_ranges.size(); ++i) @@ -1022,7 +1056,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( } ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction); - new_parts.emplace_back(part.data_part, part.alter_conversions, part.part_index_in_query, std::move(ranges_to_get_from_part)); + new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part)); } splitted_parts_and_ranges.emplace_back(std::move(new_parts)); @@ -1249,7 +1283,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( RangesInDataParts new_parts; for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) - new_parts.emplace_back(part_it->data_part, part_it->alter_conversions, part_it->part_index_in_query, part_it->ranges); + new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges); if (new_parts.empty()) continue; @@ -1362,15 +1396,14 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(bool find_exact_ranges) const { - return selectRangesToRead(prepared_parts, alter_conversions_for_parts, find_exact_ranges); + return selectRangesToRead(prepared_parts, find_exact_ranges); } -ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( - MergeTreeData::DataPartsVector parts, std::vector alter_conversions, bool find_exact_ranges) const +ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(MergeTreeData::DataPartsVector parts, bool find_exact_ranges) const { return selectRangesToRead( std::move(parts), - std::move(alter_conversions), + mutations_snapshot, storage_snapshot->metadata, query_info, context, @@ -1388,9 +1421,11 @@ static void buildIndexes( const ActionsDAG * filter_actions_dag, const MergeTreeData & data, const MergeTreeData::DataPartsVector & parts, + const MergeTreeData::MutationsSnapshotPtr & mutations_snapshot, const ContextPtr & context, const SelectQueryInfo & query_info, - const StorageMetadataPtr & metadata_snapshot) + const StorageMetadataPtr & metadata_snapshot, + const LoggerPtr & log) { indexes.reset(); @@ -1400,11 +1435,8 @@ static void buildIndexes( const auto & settings = context->getSettingsRef(); - indexes.emplace(ReadFromMergeTree::Indexes{{ - filter_actions_dag, - context, - primary_key_column_names, - primary_key.expression}, {}, {}, {}, {}, false, {}}); + indexes.emplace( + ReadFromMergeTree::Indexes{KeyCondition{filter_actions_dag, context, primary_key_column_names, primary_key.expression}}); if (metadata_snapshot->hasPartitionKey()) { @@ -1416,19 +1448,21 @@ static void buildIndexes( indexes->partition_pruner.emplace(metadata_snapshot, filter_actions_dag, context, false /* strict */); } - indexes->part_values - = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(metadata_snapshot, data, parts, filter_actions_dag, context); + indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(metadata_snapshot, data, parts, filter_actions_dag, context); MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset(indexes->part_offset_condition, filter_actions_dag, context); indexes->use_skip_indexes = settings.use_skip_indexes; - bool final = query_info.isFinal(); - - if (final && !settings.use_skip_indexes_if_final) + if (query_info.isFinal() && !settings.use_skip_indexes_if_final) indexes->use_skip_indexes = false; if (!indexes->use_skip_indexes) return; + const auto & all_indexes = metadata_snapshot->getSecondaryIndices(); + + if (all_indexes.empty()) + return; + std::unordered_set ignored_index_names; if (settings.ignore_data_skipping_indices.changed) @@ -1453,47 +1487,68 @@ static void buildIndexes( throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse ignore_data_skipping_indices ('{}')", indices); } + auto all_updated_columns = mutations_snapshot->getAllUpdatedColumns(); + UsefulSkipIndexes skip_indexes; using Key = std::pair; std::map merged; - for (const auto & index : metadata_snapshot->getSecondaryIndices()) + for (const auto & index : all_indexes) { - if (!ignored_index_names.contains(index.name)) + if (ignored_index_names.contains(index.name)) + continue; + + auto index_helper = MergeTreeIndexFactory::instance().get(index); + + if (!all_updated_columns.empty()) { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - if (index_helper->isMergeable()) + auto required_columns = index_helper->getColumnsRequiredForIndexCalc(); + auto it = std::ranges::find_if(required_columns, [&](const auto & column_name) { - auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); - if (inserted) - { - skip_indexes.merged_indices.emplace_back(); - skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); - } + return all_updated_columns.contains(column_name); + }); - skip_indexes.merged_indices[it->second].addIndex(index_helper); + if (it != required_columns.end()) + { + LOG_TRACE(log, "Index {} is not used because it depends on column {} which will be updated on fly", index.name, *it); + continue; } - else + } + + if (index_helper->isMergeable()) + { + auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); + if (inserted) { - MergeTreeIndexConditionPtr condition; - if (index_helper->isVectorSimilarityIndex()) - { + skip_indexes.merged_indices.emplace_back(); + skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); + } + + skip_indexes.merged_indices[it->second].addIndex(index_helper); + continue; + } + + MergeTreeIndexConditionPtr condition; + + if (index_helper->isVectorSimilarityIndex()) + { #if USE_USEARCH - if (const auto * vector_similarity_index = typeid_cast(index_helper.get())) - condition = vector_similarity_index->createIndexCondition(query_info, context); + if (const auto * vector_similarity_index = typeid_cast(index_helper.get())) + condition = vector_similarity_index->createIndexCondition(query_info, context); #endif - if (const auto * legacy_vector_similarity_index = typeid_cast(index_helper.get())) - condition = legacy_vector_similarity_index->createIndexCondition(query_info, context); - if (!condition) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name); - } - else - condition = index_helper->createIndexCondition(filter_actions_dag, context); + if (const auto * legacy_vector_similarity_index = typeid_cast(index_helper.get())) + condition = legacy_vector_similarity_index->createIndexCondition(query_info, context); - if (!condition->alwaysUnknownOrTrue()) - skip_indexes.useful_indices.emplace_back(index_helper, condition); - } + if (!condition) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name); + } + else + { + condition = index_helper->createIndexCondition(filter_actions_dag, context); } + + if (!condition->alwaysUnknownOrTrue()) + skip_indexes.useful_indices.emplace_back(index_helper, condition); } // move minmax indices to first positions, so they will be applied first as cheapest ones @@ -1531,15 +1586,17 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) query_info.filter_actions_dag.get(), data, prepared_parts, + mutations_snapshot, context, query_info, - storage_snapshot->metadata); + storage_snapshot->metadata, + log); } } ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( MergeTreeData::DataPartsVector parts, - std::vector alter_conversions, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info_, ContextPtr context_, @@ -1570,7 +1627,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( const Names & primary_key_column_names = primary_key.column_names; if (!indexes) - buildIndexes(indexes, query_info_.filter_actions_dag.get(), data, parts, context_, query_info_, metadata_snapshot); + buildIndexes(indexes, query_info_.filter_actions_dag.get(), data, parts, mutations_snapshot, context_, query_info_, metadata_snapshot, log); if (indexes->part_values && indexes->part_values->empty()) return std::make_shared(std::move(result)); @@ -1601,10 +1658,9 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( { MergeTreeDataSelectExecutor::filterPartsByPartition( + parts, indexes->partition_pruner, indexes->minmax_idx_condition, - parts, - alter_conversions, indexes->part_values, metadata_snapshot, data, @@ -1633,7 +1689,6 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( auto reader_settings = getMergeTreeReaderSettings(context_, query_info_); result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( std::move(parts), - std::move(alter_conversions), metadata_snapshot, context_, indexes->key_condition, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index f12da5d10bce..60707c406b31 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -110,7 +110,7 @@ class ReadFromMergeTree final : public SourceStepWithFilter ReadFromMergeTree( MergeTreeData::DataPartsVector parts_, - std::vector alter_conversions_, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot_, Names all_column_names_, const MergeTreeData & data_, const SelectQueryInfo & query_info_, @@ -121,7 +121,16 @@ class ReadFromMergeTree final : public SourceStepWithFilter std::shared_ptr max_block_numbers_to_read_, LoggerPtr log_, AnalysisResultPtr analyzed_result_ptr_, - bool enable_parallel_reading); + bool enable_parallel_reading_, + std::optional all_ranges_callback_ = std::nullopt, + std::optional read_task_callback_ = std::nullopt, + std::optional number_of_current_replica_ = std::nullopt); + + std::unique_ptr createLocalParallelReplicasReadingStep( + AnalysisResultPtr analyzed_result_ptr_, + MergeTreeAllRangesCallback all_ranges_callback_, + MergeTreeReadTaskCallback read_task_callback_, + size_t replica_number); static constexpr auto name = "ReadFromMergeTree"; String getName() const override { return name; } @@ -143,6 +152,11 @@ class ReadFromMergeTree final : public SourceStepWithFilter struct Indexes { + explicit Indexes(KeyCondition key_condition_) + : key_condition(std::move(key_condition_)) + , use_skip_indexes(false) + {} + KeyCondition key_condition; std::optional partition_pruner; std::optional minmax_idx_condition; @@ -154,7 +168,7 @@ class ReadFromMergeTree final : public SourceStepWithFilter static AnalysisResultPtr selectRangesToRead( MergeTreeData::DataPartsVector parts, - std::vector alter_conversions, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, ContextPtr context, @@ -166,8 +180,7 @@ class ReadFromMergeTree final : public SourceStepWithFilter std::optional & indexes, bool find_exact_ranges); - AnalysisResultPtr selectRangesToRead( - MergeTreeData::DataPartsVector parts, std::vector alter_conversions, bool find_exact_ranges = false) const; + AnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts, bool find_exact_ranges = false) const; AnalysisResultPtr selectRangesToRead(bool find_exact_ranges = false) const; @@ -188,7 +201,7 @@ class ReadFromMergeTree final : public SourceStepWithFilter void setAnalyzedResult(AnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); } const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; } - const std::vector & getAlterConvertionsForParts() const { return alter_conversions_for_parts; } + MergeTreeData::MutationsSnapshotPtr getMutationsSnapshot() const { return mutations_snapshot; } const MergeTreeData & getMergeTreeData() const { return data; } size_t getMaxBlockSize() const { return block_size.max_block_size_rows; } @@ -209,7 +222,7 @@ class ReadFromMergeTree final : public SourceStepWithFilter MergeTreeReaderSettings reader_settings; MergeTreeData::DataPartsVector prepared_parts; - std::vector alter_conversions_for_parts; + MergeTreeData::MutationsSnapshotPtr mutations_snapshot; Names all_column_names; @@ -269,6 +282,7 @@ class ReadFromMergeTree final : public SourceStepWithFilter std::optional read_task_callback; bool enable_vertical_final = false; bool enable_remove_parts_from_snapshot_optimization = true; + std::optional number_of_current_replica; }; } diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index cf11052cd594..3df46eb19878 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -21,8 +21,8 @@ #include #include #include -#include -#include + +#include namespace DB { @@ -362,6 +362,7 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( ASTPtr query_ast_, ClusterPtr cluster_, const StorageID & storage_id_, + ParallelReplicasReadingCoordinatorPtr coordinator_, Block header_, QueryProcessingStage::Enum stage_, ContextMutablePtr context_, @@ -369,11 +370,14 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( Scalars scalars_, Tables external_tables_, LoggerPtr log_, - std::shared_ptr storage_limits_) + std::shared_ptr storage_limits_, + std::vector pools_to_use_, + std::optional exclude_pool_index_) : ISourceStep(DataStream{.header = std::move(header_)}) , cluster(cluster_) , query_ast(query_ast_) , storage_id(storage_id_) + , coordinator(std::move(coordinator_)) , stage(std::move(stage_)) , context(context_) , throttler(throttler_) @@ -381,16 +385,24 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( , external_tables{external_tables_} , storage_limits(std::move(storage_limits_)) , log(log_) + , pools_to_use(std::move(pools_to_use_)) + , exclude_pool_index(exclude_pool_index_) { chassert(cluster->getShardCount() == 1); - std::vector description; - description.push_back(fmt::format("query: {}", formattedAST(query_ast))); + std::vector replicas; + replicas.reserve(pools_to_use.size()); - for (const auto & pool : cluster->getShardsInfo().front().per_replica_pools) - description.push_back(fmt::format("Replica: {}", pool->getHost())); + for (size_t i = 0, l = pools_to_use.size(); i < l; ++i) + { + if (exclude_pool_index.has_value() && i == exclude_pool_index) + continue; - setStepDescription(boost::algorithm::join(description, ", ")); + replicas.push_back(pools_to_use[i]->getAddress()); + } + + auto description = fmt::format("Query: {} Replicas: {}", formattedAST(query_ast), fmt::join(replicas, ", ")); + setStepDescription(std::move(description)); } void ReadFromParallelRemoteReplicasStep::enforceSorting(SortDescription output_sort_description) @@ -406,48 +418,29 @@ void ReadFromParallelRemoteReplicasStep::enforceAggregationInOrder() void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { Pipes pipes; - const Settings & current_settings = context->getSettingsRef(); - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); - const auto & shard = cluster->getShardsInfo().at(0); - size_t max_replicas_to_use = current_settings.max_parallel_replicas; - if (max_replicas_to_use > shard.getAllNodeCount()) + std::vector addresses; + addresses.reserve(pools_to_use.size()); + for (size_t i = 0, l = pools_to_use.size(); i < l; ++i) { - LOG_INFO( - getLogger("ReadFromParallelRemoteReplicasStep"), - "The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). " - "Will use the latter number to execute the query.", - current_settings.max_parallel_replicas, - shard.getAllNodeCount()); - max_replicas_to_use = shard.getAllNodeCount(); - } + if (exclude_pool_index.has_value() && i == exclude_pool_index) + continue; - std::vector shuffled_pool; - if (max_replicas_to_use < shard.getAllNodeCount()) - { - shuffled_pool = shard.pool->getShuffledPools(current_settings); - shuffled_pool.resize(max_replicas_to_use); - } - else - { - /// try to preserve replicas order if all replicas in cluster are used for query execution - /// it's important for data locality during query execution - auto priority_func = [](size_t i) { return Priority{static_cast(i)}; }; - shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func); + addresses.emplace_back(pools_to_use[i]->getAddress()); } + LOG_DEBUG(getLogger("ReadFromParallelRemoteReplicasStep"), "Addresses to use: {}", fmt::join(addresses, ", ")); - coordinator - = std::make_shared(max_replicas_to_use, current_settings.parallel_replicas_mark_segment_size); - - for (size_t i=0; i < max_replicas_to_use; ++i) + for (size_t i = 0, l = pools_to_use.size(); i < l; ++i) { - IConnections::ReplicaInfo replica_info - { + if (exclude_pool_index.has_value() && i == exclude_pool_index) + continue; + + IConnections::ReplicaInfo replica_info{ /// we should use this number specifically because efficiency of data distribution by consistent hash depends on it. .number_of_current_replica = i, }; - addPipeForSingeReplica(pipes, shuffled_pool[i].pool, replica_info); + addPipeForSingeReplica(pipes, pools_to_use[i], replica_info); } auto pipe = Pipe::unitePipes(std::move(pipes)); diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index 1adb26b2915a..74389c8f9eb8 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -70,6 +70,7 @@ class ReadFromParallelRemoteReplicasStep : public ISourceStep ASTPtr query_ast_, ClusterPtr cluster_, const StorageID & storage_id_, + ParallelReplicasReadingCoordinatorPtr coordinator_, Block header_, QueryProcessingStage::Enum stage_, ContextMutablePtr context_, @@ -77,7 +78,9 @@ class ReadFromParallelRemoteReplicasStep : public ISourceStep Scalars scalars_, Tables external_tables_, LoggerPtr log_, - std::shared_ptr storage_limits_); + std::shared_ptr storage_limits_, + std::vector pools_to_use, + std::optional exclude_pool_index_ = std::nullopt); String getName() const override { return "ReadFromRemoteParallelReplicas"; } @@ -100,6 +103,8 @@ class ReadFromParallelRemoteReplicasStep : public ISourceStep Tables external_tables; std::shared_ptr storage_limits; LoggerPtr log; + std::vector pools_to_use; + std::optional exclude_pool_index; }; } diff --git a/src/QueryPipeline/RemoteInserter.cpp b/src/QueryPipeline/RemoteInserter.cpp index 134c169e35ff..91cc25a6a17d 100644 --- a/src/QueryPipeline/RemoteInserter.cpp +++ b/src/QueryPipeline/RemoteInserter.cpp @@ -51,8 +51,9 @@ RemoteInserter::RemoteInserter( /** Send query and receive "header", that describes table structure. * Header is needed to know, what structure is required for blocks to be passed to 'write' method. */ + /// TODO (vnemkov): figure out should we pass additional roles in this case or not. connection.sendQuery( - timeouts, query, /* query_parameters */ {}, "", QueryProcessingStage::Complete, &settings, &modified_client_info, false, {}); + timeouts, query, /* query_parameters */ {}, "", QueryProcessingStage::Complete, &settings, &modified_client_info, false, /* external_roles */ {}, {}); while (true) { diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 6f8b3931803b..8d7925a34350 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -22,8 +22,12 @@ #include #include #include -#include #include +#include + +#include +#include +#include namespace ProfileEvents @@ -390,7 +394,25 @@ void RemoteQueryExecutor::sendQueryUnlocked(ClientInfo::QueryKind query_kind, As if (!duplicated_part_uuids.empty()) connections->sendIgnoredPartUUIDs(duplicated_part_uuids); - connections->sendQuery(timeouts, query, query_id, stage, modified_client_info, true); + // Collect all roles granted on this node and pass those to the remote node + std::vector local_granted_roles; + if (context->getSettingsRef().push_external_roles_in_interserver_queries && !modified_client_info.initial_user.empty()) + { + auto user = context->getAccessControl().read(modified_client_info.initial_user, false); + boost::container::flat_set granted_roles; + if (user) + { + const auto & access_control = context->getAccessControl(); + for (const auto & e : user->granted_roles.getElements()) + { + auto names = access_control.readNames(e.ids); + granted_roles.insert(names.begin(), names.end()); + } + } + local_granted_roles.insert(local_granted_roles.end(), granted_roles.begin(), granted_roles.end()); + } + + connections->sendQuery(timeouts, query, query_id, stage, modified_client_info, true, local_granted_roles); established = false; sent_query = true; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 679f72b85ff3..a0439761882a 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -57,12 +57,17 @@ #include #include +#include + #include "TCPHandler.h" #include #include +#include +#include + using namespace std::literals; using namespace DB; @@ -103,6 +108,7 @@ namespace DB::ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int UNSUPPORTED_METHOD; extern const int USER_EXPIRED; + extern const int NETWORK_ERROR; } namespace @@ -254,8 +260,8 @@ void TCPHandler::runImpl() socket().setSendTimeout(send_timeout); socket().setNoDelay(true); - in = std::make_shared(socket(), read_event); - out = std::make_shared(socket(), write_event); + in = std::make_shared(socket(), read_event); + out = std::make_shared(socket(), write_event); /// Support for PROXY protocol if (parse_proxy_protocol && !receiveProxyHeader()) @@ -280,6 +286,48 @@ void TCPHandler::runImpl() if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) receiveAddendum(); + { + /// Server side of chunked protocol negotiation. + /// Server advertises its protocol capabilities (separate for send and receive channels) by sending + /// in its 'Hello' response one of four types - chunked, notchunked, chunked_optional, notchunked_optional. + /// Not optional types are strict meaning that server only supports this type, optional means that + /// server prefer this type but capable to work in opposite. + /// Client selects which type it is going to communicate based on the settings from config or arguments, + /// and sends either "chunked" or "notchunked" protocol request in addendum section of handshake. + /// Client can detect if server's protocol capabilities are not compatible with client's settings (for example + /// server strictly requires chunked protocol but client's settings only allows notchunked protocol) - in such case + /// client should interrupt this connection. However if client continues with incompatible protocol type request, server + /// will send appropriate exception and disconnect client. + + auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) + { + bool chunked_srv = chunked_srv_str.starts_with("chunked"); + bool optional_srv = chunked_srv_str.ends_with("_optional"); + bool chunked_cl = chunked_cl_str.starts_with("chunked"); + + if (optional_srv) + return chunked_cl; + + if (chunked_cl != chunked_srv) + throw NetException( + ErrorCodes::NETWORK_ERROR, + "Incompatible protocol: {} is {}, client requested {}", + direction, + chunked_srv ? "chunked" : "notchunked", + chunked_cl ? "chunked" : "notchunked"); + + return chunked_srv; + }; + + bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "notchunked"), proto_recv_chunked_cl, "send"); + bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "notchunked"), proto_send_chunked_cl, "recv"); + + if (out_chunked) + out->enableChunked(); + if (in_chunked) + in->enableChunked(); + } + if (!is_interserver_mode) { /// If session created, then settings in session context has been updated. @@ -321,7 +369,7 @@ void TCPHandler::runImpl() { Stopwatch idle_time; UInt64 timeout_ms = std::min(poll_interval, idle_connection_timeout) * 1000000; - while (tcp_server.isOpen() && !server.isCancelled() && !static_cast(*in).poll(timeout_ms)) + while (tcp_server.isOpen() && !server.isCancelled() && !in->poll(timeout_ms)) { if (idle_time.elapsedSeconds() > idle_connection_timeout) { @@ -796,7 +844,7 @@ bool TCPHandler::readDataNext() /// We are waiting for a packet from the client. Thus, every `POLL_INTERVAL` seconds check whether we need to shut down. while (true) { - if (static_cast(*in).poll(timeout_us)) + if (in->poll(timeout_us)) { /// If client disconnected. if (in->eof()) @@ -1186,6 +1234,8 @@ void TCPHandler::processTablesStatusRequest() } response.write(*out, client_tcp_protocol_version); + + out->finishChunk(); } void TCPHandler::receiveUnexpectedTablesStatusRequest() @@ -1206,6 +1256,8 @@ void TCPHandler::sendPartUUIDs() writeVarUInt(Protocol::Server::PartUUIDs, *out); writeVectorBinary(uuids, *out); + + out->finishChunk(); out->next(); } } @@ -1214,6 +1266,8 @@ void TCPHandler::sendPartUUIDs() void TCPHandler::sendReadTaskRequestAssumeLocked() { writeVarUInt(Protocol::Server::ReadTaskRequest, *out); + + out->finishChunk(); out->next(); } @@ -1221,7 +1275,9 @@ void TCPHandler::sendReadTaskRequestAssumeLocked() void TCPHandler::sendMergeTreeAllRangesAnnouncementAssumeLocked(InitialAllRangesAnnouncement announcement) { writeVarUInt(Protocol::Server::MergeTreeAllRangesAnnouncement, *out); - announcement.serialize(*out); + announcement.serialize(*out, client_parallel_replicas_protocol_version); + + out->finishChunk(); out->next(); } @@ -1229,7 +1285,9 @@ void TCPHandler::sendMergeTreeAllRangesAnnouncementAssumeLocked(InitialAllRanges void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest request) { writeVarUInt(Protocol::Server::MergeTreeReadTaskRequest, *out); - request.serialize(*out); + request.serialize(*out, client_parallel_replicas_protocol_version); + + out->finishChunk(); out->next(); } @@ -1238,6 +1296,8 @@ void TCPHandler::sendProfileInfo(const ProfileInfo & info) { writeVarUInt(Protocol::Server::ProfileInfo, *out); info.write(*out, client_tcp_protocol_version); + + out->finishChunk(); out->next(); } @@ -1253,6 +1313,8 @@ void TCPHandler::sendTotals(const Block & totals) state.block_out->write(totals); state.maybe_compressed_out->next(); + + out->finishChunk(); out->next(); } } @@ -1269,6 +1331,8 @@ void TCPHandler::sendExtremes(const Block & extremes) state.block_out->write(extremes); state.maybe_compressed_out->next(); + + out->finishChunk(); out->next(); } } @@ -1286,6 +1350,8 @@ void TCPHandler::sendProfileEvents() writeStringBinary("", *out); state.profile_events_block_out->write(block); + + out->finishChunk(); out->next(); auto elapsed_milliseconds = stopwatch.elapsedMilliseconds(); @@ -1323,6 +1389,8 @@ void TCPHandler::sendTimezone() LOG_DEBUG(log, "TCPHandler::sendTimezone(): {}", tz); writeVarUInt(Protocol::Server::TimezoneUpdate, *out); writeStringBinary(tz, *out); + + out->finishChunk(); out->next(); } @@ -1583,6 +1651,15 @@ void TCPHandler::receiveAddendum() if (!is_interserver_mode) session->setQuotaClientKey(quota_key); + + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + readStringBinary(proto_send_chunked_cl, *in); + readStringBinary(proto_recv_chunked_cl, *in); + } + + if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL) + readVarUInt(client_parallel_replicas_protocol_version, *in); } @@ -1610,12 +1687,19 @@ void TCPHandler::sendHello() writeVarUInt(VERSION_MAJOR, *out); writeVarUInt(VERSION_MINOR, *out); writeVarUInt(DBMS_TCP_PROTOCOL_VERSION, *out); + if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL) + writeVarUInt(DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION, *out); if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) writeStringBinary(DateLUT::instance().getTimeZone(), *out); if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME) writeStringBinary(server_display_name, *out); if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSION_PATCH) writeVarUInt(VERSION_PATCH, *out); + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + writeStringBinary(server.config().getString("proto_caps.send", "notchunked"), *out); + writeStringBinary(server.config().getString("proto_caps.recv", "notchunked"), *out); + } if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES) { auto rules = server.context()->getAccessControl().getPasswordComplexityRules(); @@ -1668,6 +1752,7 @@ bool TCPHandler::receivePacket() case Protocol::Client::Ping: writeVarUInt(Protocol::Server::Pong, *out); + out->finishChunk(); out->next(); return false; @@ -1798,6 +1883,13 @@ void TCPHandler::receiveQuery() Settings passed_settings; passed_settings.read(*in, settings_format); + std::string received_extra_roles; + // TODO: check if having `is_interserver_mode` doesn't break interoperability with the CH-client. + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_INTERSERVER_EXTERNALLY_GRANTED_ROLES) + { + readStringBinary(received_extra_roles, *in); + } + /// Interserver secret. std::string received_hash; if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET) @@ -1857,6 +1949,7 @@ void TCPHandler::receiveQuery() data += state.query; data += state.query_id; data += client_info.initial_user; + data += received_extra_roles; std::string calculated_hash = encodeSHA256(data); assert(calculated_hash.size() == 32); @@ -1877,13 +1970,25 @@ void TCPHandler::receiveQuery() } else { + // In a cluster, query originator may have an access to the external auth provider (like LDAP server), + // that grants specific roles to the user. We want these roles to be granted to the user on other nodes of cluster when + // query is executed. + Strings external_roles; + if (!received_extra_roles.empty()) + { + ReadBufferFromString buffer(received_extra_roles); + + readVectorBinary(external_roles, buffer); + LOG_DEBUG(log, "Parsed extra roles [{}]", fmt::join(external_roles, ", ")); + } + LOG_DEBUG(log, "User (initial, interserver mode): {} (client: {})", client_info.initial_user, getClientAddress(client_info).toString()); /// In case of inter-server mode authorization is done with the /// initial address of the client, not the real address from which /// the query was come, since the real address is the address of /// the initiator server, while we are interested in client's /// address. - session->authenticate(AlwaysAllowCredentials{client_info.initial_user}, client_info.initial_address); + session->authenticate(AlwaysAllowCredentials{client_info.initial_user}, client_info.initial_address, external_roles); } is_interserver_authenticated = true; @@ -2197,7 +2302,7 @@ QueryState::CancellationStatus TCPHandler::getQueryCancellationStatus() after_check_cancelled.restart(); /// During request execution the only packet that can come from the client is stopping the query. - if (static_cast(*in).poll(0)) + if (in->poll(0)) { if (in->eof()) { @@ -2248,19 +2353,33 @@ void TCPHandler::sendData(const Block & block) } writeVarUInt(Protocol::Server::Data, *out); - /// Send external table name (empty name is the main table) - writeStringBinary("", *out); /// For testing hedged requests if (block.rows() > 0 && query_context->getSettingsRef().sleep_in_send_data_ms.totalMilliseconds()) { + /// This strange sequence is needed in case of chunked protocol is enabled, in order for client not to + /// hang on receiving of at least packet type - chunk will not be processed unless either chunk footer + /// or chunk continuation header is received - first 'next' is sending starting chunk containing packet type + /// and second 'next' is sending chunk continuation header. + out->next(); + /// Send external table name (empty name is the main table) + writeStringBinary("", *out); out->next(); std::chrono::milliseconds ms(query_context->getSettingsRef().sleep_in_send_data_ms.totalMilliseconds()); std::this_thread::sleep_for(ms); } + else + { + /// Send external table name (empty name is the main table) + writeStringBinary("", *out); + } state.block_out->write(block); - state.maybe_compressed_out->next(); + + if (state.maybe_compressed_out != out) + state.maybe_compressed_out->next(); + + out->finishChunk(); out->next(); } catch (...) @@ -2296,6 +2415,8 @@ void TCPHandler::sendLogData(const Block & block) writeStringBinary("", *out); state.logs_block_out->write(block); + + out->finishChunk(); out->next(); } @@ -2307,6 +2428,7 @@ void TCPHandler::sendTableColumns(const ColumnsDescription & columns) writeStringBinary("", *out); writeStringBinary(columns.toString(), *out); + out->finishChunk(); out->next(); } @@ -2316,6 +2438,8 @@ void TCPHandler::sendException(const Exception & e, bool with_stack_trace) writeVarUInt(Protocol::Server::Exception, *out); writeException(e, *out, with_stack_trace); + + out->finishChunk(); out->next(); } @@ -2326,6 +2450,8 @@ void TCPHandler::sendEndOfStream() state.io.setAllDataSent(); writeVarUInt(Protocol::Server::EndOfStream, *out); + + out->finishChunk(); out->next(); } @@ -2344,6 +2470,8 @@ void TCPHandler::sendProgress() increment.elapsed_ns = current_elapsed_ns - state.prev_elapsed_ns; state.prev_elapsed_ns = current_elapsed_ns; increment.write(*out, client_tcp_protocol_version); + + out->finishChunk(); out->next(); } diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 74afb5a14a59..3b6e0059a30d 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -18,6 +18,8 @@ #include #include #include +#include +#include #include "Core/Types.h" #include "IServer.h" @@ -186,6 +188,9 @@ class TCPHandler : public Poco::Net::TCPServerConnection UInt64 client_version_minor = 0; UInt64 client_version_patch = 0; UInt32 client_tcp_protocol_version = 0; + UInt32 client_parallel_replicas_protocol_version = 0; + String proto_send_chunked_cl = "notchunked"; + String proto_recv_chunked_cl = "notchunked"; String quota_key; /// Connection settings, which are extracted from a context. @@ -204,8 +209,8 @@ class TCPHandler : public Poco::Net::TCPServerConnection ClientInfo::QueryKind query_kind = ClientInfo::QueryKind::NO_QUERY; /// Streams for reading/writing from/to client connection socket. - std::shared_ptr in; - std::shared_ptr out; + std::shared_ptr in; + std::shared_ptr out; ProfileEvents::Event read_event; ProfileEvents::Event write_event; diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 1de8ccce2a68..7f368102dfd3 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -273,6 +273,8 @@ ConnectionPoolWithFailoverPtr DistributedAsyncInsertDirectoryQueue::createPool(c address.default_database, address.user, address.password, + address.proto_send_chunked, + address.proto_recv_chunked, address.quota_key, address.cluster, address.cluster_secret, diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index b5e32fc9593b..1a57f780f65d 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -271,6 +271,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo /// Return true if the trivial count query could be optimized without reading the data at all /// in totalRows() or totalRowsByPartitionPredicate() methods or with optimized reading in read() method. + /// 'storage_snapshot' may be nullptr. virtual bool supportsTrivialCountOptimization(const StorageSnapshotPtr & /*storage_snapshot*/, ContextPtr /*query_context*/) const { return false; diff --git a/src/Storages/MergeTree/AlterConversions.cpp b/src/Storages/MergeTree/AlterConversions.cpp index 31f8f17e2c14..a36611e3d87f 100644 --- a/src/Storages/MergeTree/AlterConversions.cpp +++ b/src/Storages/MergeTree/AlterConversions.cpp @@ -9,9 +9,15 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -bool AlterConversions::supportsMutationCommandType(MutationCommand::Type t) +bool AlterConversions::isSupportedDataMutation(MutationCommand::Type) { - return t == MutationCommand::Type::RENAME_COLUMN; + /// Currently there is no such mutations. See setting 'apply_mutations_on_fly'. + return false; +} + +bool AlterConversions::isSupportedMetadataMutation(MutationCommand::Type type) +{ + return type == MutationCommand::Type::RENAME_COLUMN; } void AlterConversions::addMutationCommand(const MutationCommand & command) diff --git a/src/Storages/MergeTree/AlterConversions.h b/src/Storages/MergeTree/AlterConversions.h index 7bcd4af8b36d..046cc1d2491f 100644 --- a/src/Storages/MergeTree/AlterConversions.h +++ b/src/Storages/MergeTree/AlterConversions.h @@ -1,8 +1,8 @@ #pragma once #include -#include -#include +#include +#include namespace DB @@ -11,11 +11,17 @@ namespace DB /// Alter conversions which should be applied on-fly for part. /// Built from of the most recent mutation commands for part. /// Now only ALTER RENAME COLUMN is applied. -class AlterConversions : private boost::noncopyable +class AlterConversions : private WithContext, boost::noncopyable { public: AlterConversions() = default; + AlterConversions(StorageMetadataPtr metadata_snapshot_, ContextPtr context_) + : WithContext(context_) + , metadata_snapshot(std::move(metadata_snapshot_)) + { + } + struct RenamePair { std::string rename_to; @@ -34,11 +40,13 @@ class AlterConversions : private boost::noncopyable /// Get column old name before rename (lookup by key in rename_map) std::string getColumnOldName(const std::string & new_name) const; - static bool supportsMutationCommandType(MutationCommand::Type); + static bool isSupportedDataMutation(MutationCommand::Type type); + static bool isSupportedMetadataMutation(MutationCommand::Type type); private: /// Rename map new_name -> old_name. std::vector rename_map; + StorageMetadataPtr metadata_snapshot; }; using AlterConversionsPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 7d3c2819d8dc..0127a3b7ae63 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -274,6 +274,14 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() if (enabledBlockOffsetColumn(global_ctx)) addGatheringColumn(global_ctx, BlockOffsetColumn::name, BlockOffsetColumn::type); + MergeTreeData::IMutationsSnapshot::Params params + { + .metadata_version = global_ctx->metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = MergeTreeData::getMinMetadataVersion(global_ctx->future_part->parts), + }; + + auto mutations_snapshot = global_ctx->data->getMutationsSnapshot(params); + SerializationInfo::Settings info_settings = { .ratio_of_defaults_for_sparse = global_ctx->data->getSettings()->ratio_of_defaults_for_sparse_serialization, @@ -281,10 +289,12 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() }; SerializationInfoByName infos(global_ctx->storage_columns, info_settings); + global_ctx->alter_conversions.reserve(global_ctx->future_part->parts.size()); for (const auto & part : global_ctx->future_part->parts) { global_ctx->new_data_part->ttl_infos.update(part->ttl_infos); + if (global_ctx->metadata_snapshot->hasAnyTTL() && !part->checkAllTTLCalculated(global_ctx->metadata_snapshot)) { LOG_INFO(ctx->log, "Some TTL values were not calculated for part {}. Will calculate them forcefully during merge.", part->name); @@ -305,6 +315,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() infos.add(part_infos); } + + global_ctx->alter_conversions.push_back(MergeTreeData::getAlterConversionsForPart(part, mutations_snapshot, global_ctx->metadata_snapshot, global_ctx->context)); } const auto & local_part_min_ttl = global_ctx->new_data_part->ttl_infos.part_min_ttl; @@ -657,6 +669,7 @@ Pipe MergeTask::VerticalMergeStage::createPipeForReadingOneColumn(const String & *global_ctx->data, global_ctx->storage_snapshot, global_ctx->future_part->parts[part_num], + global_ctx->alter_conversions[part_num], Names{column_name}, /*mark_ranges=*/ {}, global_ctx->input_rows_filtered, @@ -1107,13 +1120,14 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() global_ctx->horizontal_stage_progress = std::make_unique( ctx->column_sizes ? ctx->column_sizes->keyColumnsWeight() : 1.0); - for (const auto & part : global_ctx->future_part->parts) + for (size_t i = 0; i < global_ctx->future_part->parts.size(); ++i) { Pipe pipe = createMergeTreeSequentialSource( MergeTreeSequentialSourceType::Merge, *global_ctx->data, global_ctx->storage_snapshot, - part, + global_ctx->future_part->parts[i], + global_ctx->alter_conversions[i], global_ctx->merging_columns.getNames(), /*mark_ranges=*/ {}, global_ctx->input_rows_filtered, diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 2871e8c0f4a7..7e8969145d5d 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -162,6 +162,7 @@ class MergeTask StorageSnapshotPtr storage_snapshot{nullptr}; StorageMetadataPtr metadata_snapshot{nullptr}; FutureMergedMutatedPartPtr future_part{nullptr}; + std::vector alter_conversions; /// This will be either nullptr or new_data_part, so raw pointer is ok. IMergeTreeDataPart * parent_part{nullptr}; ContextPtr context{nullptr}; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 42214891db8c..76215c0f8673 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7168,11 +7168,16 @@ UInt64 MergeTreeData::estimateNumberOfRowsToRead( ContextPtr query_context, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info) const { const auto & snapshot_data = assert_cast(*storage_snapshot->data); - const auto & parts = snapshot_data.parts; MergeTreeDataSelectExecutor reader(*this); auto result_ptr = reader.estimateNumMarksToRead( - parts, {}, storage_snapshot->metadata, query_info, query_context, query_context->getSettingsRef().max_threads); + snapshot_data.parts, + snapshot_data.mutations_snapshot, + storage_snapshot->metadata->getColumns().getAll().getNames(), + storage_snapshot->metadata, + query_info, + query_context, + query_context->getSettingsRef().max_threads); UInt64 total_rows = result_ptr->selected_rows; if (query_info.trivial_limit > 0 && query_info.trivial_limit < total_rows) @@ -8202,11 +8207,15 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S return true; } -AlterConversionsPtr MergeTreeData::getAlterConversionsForPart(MergeTreeDataPartPtr part) const +AlterConversionsPtr MergeTreeData::getAlterConversionsForPart( + const MergeTreeDataPartPtr & part, + const MutationsSnapshotPtr & mutations, + const StorageMetadataPtr & metadata, + const ContextPtr & query_context) { - auto commands = getAlterMutationCommandsForPart(part); + auto commands = mutations->getAlterMutationCommandsForPart(part); + auto result = std::make_shared(metadata, query_context); - auto result = std::make_shared(); for (const auto & command : commands | std::views::reverse) result->addMutationCommand(command); @@ -8498,9 +8507,28 @@ void MergeTreeData::updateObjectColumns(const DataPartPtr & part, const DataPart DB::updateObjectColumns(object_columns, columns, part->getColumns()); } -bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const +bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr query_context) const { - return !hasLightweightDeletedMask(); + if (hasLightweightDeletedMask()) + return false; + + if (!storage_snapshot) + return !query_context->getSettingsRef().apply_mutations_on_fly; + + const auto & snapshot_data = assert_cast(*storage_snapshot->data); + return !snapshot_data.mutations_snapshot->hasDataMutations(); +} + +Int64 MergeTreeData::getMinMetadataVersion(const DataPartsVector & parts) +{ + Int64 version = -1; + for (const auto & part : parts) + { + Int64 part_version = part->getMetadataVersion(); + if (version == -1 || part_version < version) + version = part_version; + } + return version; } StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const @@ -8514,10 +8542,14 @@ StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & object_columns_copy = object_columns; } - snapshot_data->alter_conversions.reserve(snapshot_data->parts.size()); - for (const auto & part : snapshot_data->parts) - snapshot_data->alter_conversions.push_back(getAlterConversionsForPart(part)); + IMutationsSnapshot::Params params + { + .metadata_version = metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = getMinMetadataVersion(snapshot_data->parts), + .need_data_mutations = query_context->getSettingsRef().apply_mutations_on_fly, + }; + snapshot_data->mutations_snapshot = getMutationsSnapshot(params); return std::make_shared(*this, metadata_snapshot, std::move(object_columns_copy), std::move(snapshot_data)); } @@ -8735,28 +8767,57 @@ void MergeTreeData::verifySortingKey(const KeyDescription & sorting_key) } } -bool updateAlterConversionsMutations(const MutationCommands & commands, std::atomic & alter_conversions_mutations, bool remove) +static void updateMutationsCounters( + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, + const MutationCommands & commands, + Int64 increment) { + if (num_data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", num_data_mutations_to_apply); + + if (num_metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", num_metadata_mutations_to_apply); + + bool has_data_mutation = false; + bool has_metadata_mutation = false; + for (const auto & command : commands) { - if (AlterConversions::supportsMutationCommandType(command.type)) + if (!has_data_mutation && AlterConversions::isSupportedDataMutation(command.type)) { - if (remove) - { - --alter_conversions_mutations; - if (alter_conversions_mutations < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly mutations counter is negative ({})", alter_conversions_mutations); - } - else - { - if (alter_conversions_mutations < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly mutations counter is negative ({})", alter_conversions_mutations); - ++alter_conversions_mutations; - } - return true; + num_data_mutations_to_apply += increment; + has_data_mutation = true; + + if (num_data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", num_data_mutations_to_apply); + } + + if (!has_metadata_mutation && AlterConversions::isSupportedMetadataMutation(command.type)) + { + num_metadata_mutations_to_apply += increment; + has_metadata_mutation = true; + + if (num_metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", num_metadata_mutations_to_apply); } } - return false; +} + +void incrementMutationsCounters( + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, + const MutationCommands & commands) +{ + updateMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, 1); +} + +void decrementMutationsCounters( + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, + const MutationCommands & commands) +{ + updateMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, -1); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 6662df3db84b..dc37d5e7dadf 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -443,14 +443,53 @@ class MergeTreeData : public IStorage, public WithMutableContext bool areAsynchronousInsertsEnabled() const override; - bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override; + bool supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr query_context) const override; + + /// A snapshot of pending mutations that weren't applied to some of the parts yet + /// and should be applied on the fly (i.e. when reading from the part). + /// Mutations not supported by AlterConversions (supportsMutationCommandType()) can be omitted. + struct IMutationsSnapshot + { + /// Contains info that doesn't depend on state of mutations. + struct Params + { + Int64 metadata_version = -1; + Int64 min_part_metadata_version = -1; + bool need_data_mutations = false; + }; + + /// Contains info that depends on state of mutations. + struct Info + { + Int64 num_data_mutations = 0; + Int64 num_metadata_mutations = 0; + }; + + Params params; + Info info; + + IMutationsSnapshot() = default; + IMutationsSnapshot(Params params_, Info info_): params(std::move(params_)), info(std::move(info_)) {} + + /// Returns mutation commands that are required to be applied to the `part`. + /// @return list of mutation commands, in *reverse* order (newest to oldest) + virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; + virtual std::shared_ptr cloneEmpty() const = 0; + virtual NameSet getAllUpdatedColumns() const = 0; + + bool hasDataMutations() const { return params.need_data_mutations && info.num_data_mutations > 0; } + + virtual ~IMutationsSnapshot() = default; + }; + + using MutationsSnapshotPtr = std::shared_ptr; /// Snapshot for MergeTree contains the current set of data parts - /// at the moment of the start of query. + /// and mutations required to be applied at the moment of the start of query. struct SnapshotData : public StorageSnapshot::Data { DataPartsVector parts; - std::vector alter_conversions; + MutationsSnapshotPtr mutations_snapshot; }; StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override; @@ -929,8 +968,18 @@ class MergeTreeData : public IStorage, public WithMutableContext Disks getDisks() const { return getStoragePolicy()->getDisks(); } + /// Returns a snapshot of mutations that probably will be applied on the fly to parts during reading. + virtual MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const = 0; + + /// Returns the minimum version of metadata among parts. + static Int64 getMinMetadataVersion(const DataPartsVector & parts); + /// Return alter conversions for part which must be applied on fly. - AlterConversionsPtr getAlterConversionsForPart(MergeTreeDataPartPtr part) const; + static AlterConversionsPtr getAlterConversionsForPart( + const MergeTreeDataPartPtr & part, + const MutationsSnapshotPtr & mutations, + const StorageMetadataPtr & metadata, + const ContextPtr & query_context); /// Returns destination disk or volume for the TTL rule according to current storage policy. SpacePtr getDestinationForMoveTTL(const TTLDescription & move_ttl) const; @@ -1450,13 +1499,6 @@ class MergeTreeData : public IStorage, public WithMutableContext /// mechanisms for parts locking virtual bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const = 0; - /// Return pending mutations that weren't applied to `part` yet and should be applied on the fly - /// (i.e. when reading from the part). Mutations not supported by AlterConversions - /// (supportsMutationCommandType()) can be omitted. - /// - /// @return list of mutations, in *reverse* order (newest to oldest) - virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; - struct PartBackupEntries { String part_name; @@ -1731,7 +1773,14 @@ struct CurrentlySubmergingEmergingTagger }; /// Look at MutationCommands if it contains mutations for AlterConversions, update the counter. -/// Return true if the counter had been updated -bool updateAlterConversionsMutations(const MutationCommands & commands, std::atomic & alter_conversions_mutations, bool remove); +void incrementMutationsCounters( + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, + const MutationCommands & commands); + +void decrementMutationsCounters( + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, + const MutationCommands & commands); } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index bf12b3966d33..f545e2e054c6 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -133,12 +133,10 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( bool enable_parallel_reading) const { const auto & snapshot_data = assert_cast(*storage_snapshot->data); - const auto & parts = snapshot_data.parts; - const auto & alter_conversions = snapshot_data.alter_conversions; auto step = readFromParts( - parts, - alter_conversions, + snapshot_data.parts, + snapshot_data.mutations_snapshot, column_names_to_return, storage_snapshot, query_info, @@ -271,10 +269,14 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( * It is also important that the entire universe can be covered using SAMPLE 0.1 OFFSET 0, ... OFFSET 0.9 and similar decimals. */ - auto parallel_replicas_mode = context->getParallelReplicasMode(); + const bool can_use_sampling_key_parallel_replicas = + settings.allow_experimental_parallel_reading_from_replicas > 0 + && settings.max_parallel_replicas > 1 + && settings.parallel_replicas_mode == ParallelReplicasMode::SAMPLING_KEY; + /// Parallel replicas has been requested but there is no way to sample data. /// Select all data from first replica and no data from other replicas. - if (settings.parallel_replicas_count > 1 && parallel_replicas_mode == Context::ParallelReplicasMode::SAMPLE_KEY + if (can_use_sampling_key_parallel_replicas && settings.parallel_replicas_count > 1 && !data.supportsSampling() && settings.parallel_replica_offset > 0) { LOG_DEBUG( @@ -285,9 +287,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( return sampling; } - sampling.use_sampling = relative_sample_size > 0 - || (settings.parallel_replicas_count > 1 && parallel_replicas_mode == Context::ParallelReplicasMode::SAMPLE_KEY - && data.supportsSampling()); + sampling.use_sampling = relative_sample_size > 0 || (can_use_sampling_key_parallel_replicas && settings.parallel_replicas_count > 1 && data.supportsSampling()); bool no_data = false; /// There is nothing left after sampling. if (sampling.use_sampling) @@ -494,10 +494,9 @@ std::optional> MergeTreeDataSelectExecutor::filterPar } void MergeTreeDataSelectExecutor::filterPartsByPartition( + MergeTreeData::DataPartsVector & parts, const std::optional & partition_pruner, const std::optional & minmax_idx_condition, - MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, @@ -506,8 +505,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( LoggerPtr log, ReadFromMergeTree::IndexStats & index_stats) { - chassert(alter_conversions.empty() || parts.size() == alter_conversions.size()); - const Settings & settings = context->getSettingsRef(); DataTypes minmax_columns_types; @@ -531,7 +528,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( if (query_context->getSettingsRef().allow_experimental_query_deduplication) selectPartsToReadWithUUIDFilter( parts, - alter_conversions, part_values, data.getPinnedPartUUIDs(), minmax_idx_condition, @@ -544,7 +540,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( else selectPartsToRead( parts, - alter_conversions, part_values, minmax_idx_condition, minmax_columns_types, @@ -583,7 +578,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, - std::vector && alter_conversions, StorageMetadataPtr metadata_snapshot, const ContextPtr & context, const KeyCondition & key_condition, @@ -596,8 +590,6 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd bool use_skip_indexes, bool find_exact_ranges) { - chassert(alter_conversions.empty() || parts.size() == alter_conversions.size()); - RangesInDataParts parts_with_ranges; parts_with_ranges.resize(parts.size()); const Settings & settings = context->getSettingsRef(); @@ -656,11 +648,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd auto process_part = [&](size_t part_index) { auto & part = parts[part_index]; - auto alter_conversions_for_part = !alter_conversions.empty() - ? alter_conversions[part_index] - : std::make_shared(); - RangesInDataPart ranges(part, alter_conversions_for_part, part_index); + RangesInDataPart ranges(part, part_index); size_t total_marks_count = part->index_granularity.getMarksCountWithoutFinal(); if (metadata_snapshot->hasPrimaryKey() || part_offset_condition) @@ -898,6 +887,7 @@ std::shared_ptr MergeTreeDataSelectExecutor::checkLimits( ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMarksToRead( MergeTreeData::DataPartsVector parts, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const Names & column_names_to_return, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -910,11 +900,9 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar return std::make_shared(); std::optional indexes; - /// NOTE: We don't need alter_conversions because the returned analysis_result is only used for: - /// 1. estimate the number of rows to read; 2. projection reading, which doesn't have alter_conversions. return ReadFromMergeTree::selectRangesToRead( std::move(parts), - /*alter_conversions=*/{}, + mutations_snapshot, metadata_snapshot, query_info, context, @@ -929,7 +917,7 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( MergeTreeData::DataPartsVector parts, - std::vector alter_conversions, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const Names & column_names_to_return, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info, @@ -951,7 +939,7 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( return std::make_unique( std::move(parts), - std::move(alter_conversions), + std::move(mutations_snapshot), column_names_to_return, data, query_info, @@ -1548,7 +1536,6 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingMergedIndex( void MergeTreeDataSelectExecutor::selectPartsToRead( MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, @@ -1557,14 +1544,11 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( PartFilterCounters & counters) { MergeTreeData::DataPartsVector prev_parts; - std::vector prev_conversions; - std::swap(prev_parts, parts); - std::swap(prev_conversions, alter_conversions); - for (size_t i = 0; i < prev_parts.size(); ++i) + for (const auto & part_or_projection : prev_parts) { - const auto * part = prev_parts[i]->isProjectionPart() ? prev_parts[i]->getParentPart() : prev_parts[i].get(); + const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get(); if (part_values && part_values->find(part->name) == part_values->end()) continue; @@ -1601,15 +1585,12 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( counters.num_parts_after_partition_pruner += 1; counters.num_granules_after_partition_pruner += num_granules; - parts.push_back(prev_parts[i]); - if (!prev_conversions.empty()) - alter_conversions.push_back(prev_conversions[i]); + parts.push_back(part_or_projection); } } void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, @@ -1622,22 +1603,17 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( { /// process_parts prepare parts that have to be read for the query, /// returns false if duplicated parts' UUID have been met - auto select_parts = [&] ( - MergeTreeData::DataPartsVector & selected_parts, - std::vector & selected_conversions) -> bool + auto select_parts = [&](MergeTreeData::DataPartsVector & selected_parts) -> bool { auto ignored_part_uuids = query_context->getIgnoredPartUUIDs(); std::unordered_set temp_part_uuids; MergeTreeData::DataPartsVector prev_parts; - std::vector prev_conversions; - std::swap(prev_parts, selected_parts); - std::swap(prev_conversions, selected_conversions); - for (size_t i = 0; i < prev_parts.size(); ++i) + for (const auto & part_or_projection : prev_parts) { - const auto * part = prev_parts[i]->isProjectionPart() ? prev_parts[i]->getParentPart() : prev_parts[i].get(); + const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get(); if (part_values && part_values->find(part->name) == part_values->end()) continue; @@ -1687,9 +1663,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( throw Exception(ErrorCodes::LOGICAL_ERROR, "Found a part with the same UUID on the same replica."); } - selected_parts.push_back(prev_parts[i]); - if (!prev_conversions.empty()) - selected_conversions.push_back(prev_conversions[i]); + selected_parts.push_back(part_or_projection); } if (!temp_part_uuids.empty()) @@ -1708,7 +1682,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( }; /// Process parts that have to be read for a query. - auto needs_retry = !select_parts(parts, alter_conversions); + auto needs_retry = !select_parts(parts); /// If any duplicated part UUIDs met during the first step, try to ignore them in second pass. /// This may happen when `prefer_localhost_replica` is set and "distributed" stage runs in the same process with "remote" stage. @@ -1719,7 +1693,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( counters = PartFilterCounters(); /// Second attempt didn't help, throw an exception - if (!select_parts(parts, alter_conversions)) + if (!select_parts(parts)) throw Exception(ErrorCodes::DUPLICATED_PART_UUIDS, "Found duplicate UUIDs while processing query."); } } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 39bff5eacd6a..3668eb0ad90a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -40,7 +40,7 @@ class MergeTreeDataSelectExecutor /// The same as read, but with specified set of parts. QueryPlanStepPtr readFromParts( MergeTreeData::DataPartsVector parts, - std::vector alter_conversions, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info, @@ -56,6 +56,7 @@ class MergeTreeDataSelectExecutor /// This method is used to select best projection for table. ReadFromMergeTree::AnalysisResultPtr estimateNumMarksToRead( MergeTreeData::DataPartsVector parts, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -120,7 +121,6 @@ class MergeTreeDataSelectExecutor /// as well as `max_block_number_to_read`. static void selectPartsToRead( MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, @@ -131,7 +131,6 @@ class MergeTreeDataSelectExecutor /// Same as previous but also skip parts uuids if any to the query context, or skip parts which uuids marked as excluded. static void selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, @@ -175,10 +174,9 @@ class MergeTreeDataSelectExecutor /// Filter parts using minmax index and partition key. static void filterPartsByPartition( + MergeTreeData::DataPartsVector & parts, const std::optional & partition_pruner, const std::optional & minmax_idx_condition, - MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, @@ -192,7 +190,6 @@ class MergeTreeDataSelectExecutor /// If 'check_limits = true' it will throw exception if the amount of data exceed the limits from settings. static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, - std::vector && alter_conversions, StorageMetadataPtr metadata_snapshot, const ContextPtr & context, const KeyCondition & key_condition, diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 4dbccb916202..5970aed497e0 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -50,7 +50,7 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, const TransactionID & tid_, const WriteSettings & settings) : create_time(time(nullptr)) - , commands(std::move(commands_)) + , commands(std::make_shared(std::move(commands_))) , disk(std::move(disk_)) , path_prefix(path_prefix_) , file_name("tmp_mutation_" + toString(tmp_number) + ".txt") @@ -63,7 +63,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP *out << "format version: 1\n" << "create time: " << LocalDateTime(create_time, DateLUT::serverTimezoneInstance()) << "\n"; *out << "commands: "; - commands.writeText(*out, /* with_pure_metadata_commands = */ false); + commands->writeText(*out, /* with_pure_metadata_commands = */ false); *out << "\n"; if (tid.isPrehistoric()) { @@ -116,7 +116,8 @@ void MergeTreeMutationEntry::writeCSN(CSN csn_) } MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & path_prefix_, const String & file_name_) - : disk(std::move(disk_)) + : commands(std::make_shared()) + , disk(std::move(disk_)) , path_prefix(path_prefix_) , file_name(file_name_) , is_temp(false) @@ -133,7 +134,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); *buf >> "commands: "; - commands.readText(*buf); + commands->readText(*buf); *buf >> "\n"; if (buf->eof()) @@ -177,7 +178,7 @@ std::shared_ptr MergeTreeMutationEntry::backup() const out << "block number: " << block_number << "\n"; out << "commands: "; - commands.writeText(out, /* with_pure_metadata_commands = */ false); + commands->writeText(out, /* with_pure_metadata_commands = */ false); out << "\n"; return std::make_shared(out.str()); diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 04297f2852a1..f41ad2a17f85 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -16,7 +16,7 @@ class IBackupEntry; struct MergeTreeMutationEntry { time_t create_time = 0; - MutationCommands commands; + std::shared_ptr commands; DiskPtr disk; String path_prefix; diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index a9b77fb6c034..09bbf33ba9b3 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -85,6 +85,7 @@ MergeTreeReadTask::Readers MergeTreePrefetchedReadPool::PrefetchedReaders::get() MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -95,6 +96,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, @@ -103,7 +105,6 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( column_names_, settings_, context_) - , WithContext(context_) , prefetch_threadpool(getContext()->getPrefetchThreadpool()) , log(getLogger("MergeTreePrefetchedReadPool(" + (parts_ranges.empty() ? "" : parts_ranges.front().data_part->storage.getStorageID().getNameForLogs()) + ")")) { diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index a3a572276301..1a7092509378 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -14,11 +14,12 @@ using MergeTreeReaderPtr = std::unique_ptr; /// A class which is responsible for creating read tasks /// which are later taken by readers via getTask method. /// Does prefetching for the read tasks it creates. -class MergeTreePrefetchedReadPool : public MergeTreeReadPoolBase, private WithContext +class MergeTreePrefetchedReadPool : public MergeTreeReadPoolBase { public: MergeTreePrefetchedReadPool( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index cc321cd5a4da..97acd917d8ba 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -28,6 +28,7 @@ extern const int LOGICAL_ERROR; MergeTreeReadPool::MergeTreeReadPool( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -38,6 +39,7 @@ MergeTreeReadPool::MergeTreeReadPool( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index 7f0de21e1a46..c51dca315f99 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -26,6 +26,7 @@ class MergeTreeReadPool : public MergeTreeReadPoolBase MergeTreeReadPool( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 6d2560bc9c79..021b340d7467 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -4,9 +4,6 @@ #include #include -#include - - namespace DB { @@ -17,6 +14,7 @@ namespace ErrorCodes MergeTreeReadPoolBase::MergeTreeReadPoolBase( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -25,7 +23,9 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( const Names & column_names_, const PoolSettings & pool_settings_, const ContextPtr & context_) - : parts_ranges(std::move(parts_)) + : WithContext(context_) + , parts_ranges(std::move(parts_)) + , mutations_snapshot(std::move(mutations_snapshot_)) , shared_virtual_fields(std::move(shared_virtual_fields_)) , storage_snapshot(storage_snapshot_) , prewhere_info(prewhere_info_) @@ -119,9 +119,9 @@ void MergeTreeReadPoolBase::fillPerPartInfos(const Settings & settings) } read_task_info.part_index_in_query = part_with_ranges.part_index_in_query; - read_task_info.alter_conversions = part_with_ranges.alter_conversions; + read_task_info.alter_conversions = MergeTreeData::getAlterConversionsForPart(part_with_ranges.data_part, mutations_snapshot, storage_snapshot->metadata, getContext()); - LoadedMergeTreeDataPartInfoForReader part_info(part_with_ranges.data_part, part_with_ranges.alter_conversions); + LoadedMergeTreeDataPartInfoForReader part_info(part_with_ranges.data_part, read_task_info.alter_conversions); read_task_info.task_columns = getReadTaskColumns( part_info, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.h b/src/Storages/MergeTree/MergeTreeReadPoolBase.h index 123f7538ba8d..7f9106d476ef 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.h @@ -6,9 +6,11 @@ namespace DB { -class MergeTreeReadPoolBase : public IMergeTreeReadPool +class MergeTreeReadPoolBase : public IMergeTreeReadPool, protected WithContext { public: + using MutationsSnapshotPtr = MergeTreeData::MutationsSnapshotPtr; + struct PoolSettings { size_t threads = 0; @@ -23,6 +25,7 @@ class MergeTreeReadPoolBase : public IMergeTreeReadPool MergeTreeReadPoolBase( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -37,6 +40,7 @@ class MergeTreeReadPoolBase : public IMergeTreeReadPool protected: /// Initialized in constructor const RangesInDataParts parts_ranges; + const MutationsSnapshotPtr mutations_snapshot; const VirtualFields shared_virtual_fields; const StorageSnapshotPtr storage_snapshot; const PrewhereInfoPtr prewhere_info; diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp index 4c0391ffa571..60f127acdaed 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp @@ -12,6 +12,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( bool has_limit_below_one_block_, MergeTreeReadType read_type_, RangesInDataParts parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -22,6 +23,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h index 9fedf396a6bd..a3668acb1707 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h @@ -11,6 +11,7 @@ class MergeTreeReadPoolInOrder : public MergeTreeReadPoolBase bool has_limit_below_one_block_, MergeTreeReadType read_type_, RangesInDataParts parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index 33eaf5a49bd9..d138bed00c79 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -1,18 +1,106 @@ -#include #include +#include +#include + +#include +#include +#include + + +namespace +{ + +size_t chooseSegmentSize( + LoggerPtr log, size_t mark_segment_size, size_t min_marks_per_task, size_t threads, size_t sum_marks, size_t number_of_replicas) +{ + /// Mark segment size determines the granularity of work distribution between replicas. + /// Namely, coordinator will take mark segments of size `mark_segment_size` granules, calculate hash of this segment and assign it to corresponding replica. + /// Small segments are good when we read a small random subset of a table, big - when we do full-scan over a large table. + /// With small segments there is a problem: consider a query like `select max(time) from wikistat`. Average size of `time` per granule is ~5KB. So when we + /// read 128 granules we still read only ~0.5MB of data. With default fs cache segment size of 4MB it means a lot of data will be downloaded and written + /// in cache for no reason. General case will look like this: + /// + /// +---------- useful data + /// v + /// +------+--+------+ + /// |------|++| | + /// |------|++| | + /// +------+--+------+ + /// ^ + /// predownloaded data -----------+ + /// + /// Having large segments solves all the problems in this case. Also bigger segments mean less requests (especially for big tables and full-scans). + /// These three values below chosen mostly intuitively. 128 granules is 1M rows - just a good starting point, 16384 seems to still make sense when reading + /// billions of rows and 1024 - is a reasonable point in between. We limit our choice to only these three options because when we change segment size + /// we essentially change distribution of data between replicas and of course we don't want to use simultaneously tens of different distributions, because + /// it would be a huge waste of cache space. + constexpr std::array borders{128, 1024, 16384}; + + LOG_DEBUG( + log, + "mark_segment_size={}, min_marks_per_task*threads={}, sum_marks/number_of_replicas^2={}", + mark_segment_size, + min_marks_per_task * threads, + sum_marks / number_of_replicas / number_of_replicas); + + /// Here we take max of two numbers: + /// * (min_marks_per_task * threads) = the number of marks we request from the coordinator each time - there is no point to have segments smaller than one unit of work for a replica + /// * (sum_marks / number_of_replicas^2) - we use consistent hashing for work distribution (including work stealing). If we have a really slow replica + /// everything except (1/number_of_replicas) portion of its work will be stolen by other replicas. And it owns (1/number_of_replicas) share of total number of marks. + /// Also important to note here that sum_marks is calculated after PK analysis, it means in particular that different segment sizes might be used for the + /// same table for different queries (it is intentional). + /// + /// Positive `mark_segment_size` means it is a user provided value, we have to preserve it. + if (mark_segment_size == 0) + mark_segment_size = std::max(min_marks_per_task * threads, sum_marks / number_of_replicas / number_of_replicas); + + /// Squeeze the value to the borders. + mark_segment_size = std::clamp(mark_segment_size, borders.front(), borders.back()); + /// After we calculated a hopefully good value for segment_size let's just find the maximal border that is not bigger than the chosen value. + for (auto border : borders | std::views::reverse) + { + if (mark_segment_size >= border) + { + LOG_DEBUG(log, "Chosen segment size: {}", border); + return border; + } + } + + UNREACHABLE(); +} + +size_t getMinMarksPerTask(size_t min_marks_per_task, const std::vector & per_part_infos) +{ + for (const auto & info : per_part_infos) + min_marks_per_task = std::max(min_marks_per_task, info->min_marks_per_task); + + if (min_marks_per_task == 0) + throw DB::Exception( + DB::ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)"); + + return min_marks_per_task; +} +} + +namespace ProfileEvents +{ +extern const Event ParallelReplicasReadMarks; +} namespace DB { namespace ErrorCodes { - extern const int LOGICAL_ERROR; +extern const int LOGICAL_ERROR; +extern const int BAD_ARGUMENTS; } MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( ParallelReadingExtension extension_, RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -23,6 +111,7 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, @@ -33,13 +122,16 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( context_) , extension(std::move(extension_)) , coordination_mode(CoordinationMode::Default) - , min_marks_per_task(pool_settings.min_marks_for_concurrent_read) + , min_marks_per_task(getMinMarksPerTask(pool_settings.min_marks_for_concurrent_read, per_part_infos)) + , mark_segment_size(chooseSegmentSize( + log, + context_->getSettingsRef().parallel_replicas_mark_segment_size, + min_marks_per_task, + pool_settings.threads, + pool_settings.sum_marks, + extension.getTotalNodesCount())) { - for (const auto & info : per_part_infos) - min_marks_per_task = std::max(min_marks_per_task, info->min_marks_per_task); - - extension.all_callback( - InitialAllRangesAnnouncement(coordination_mode, parts_ranges.getDescriptions(), extension.number_of_current_replica)); + extension.sendInitialRequest(coordination_mode, parts_ranges, mark_segment_size); } MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t /*task_idx*/, MergeTreeReadTask * previous_task) @@ -51,12 +143,11 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t /*task_id if (buffered_ranges.empty()) { - auto result = extension.callback(ParallelReadRequest( + auto result = extension.sendReadRequest( coordination_mode, - extension.number_of_current_replica, min_marks_per_task * pool_settings.threads, /// For Default coordination mode we don't need to pass part names. - RangesInDataPartsDescription{})); + RangesInDataPartsDescription{}); if (!result || result->finish) { @@ -104,6 +195,7 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t /*task_id if (current_task.ranges.empty()) buffered_ranges.pop_front(); + ProfileEvents::increment(ProfileEvents::ParallelReplicasReadMarks, current_sum_marks); return createTask(per_part_infos[part_idx], std::move(ranges_to_read), previous_task); } diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h index 6ba63cc2c9a0..b9f2e133c4a2 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h @@ -11,6 +11,7 @@ class MergeTreeReadPoolParallelReplicas : public MergeTreeReadPoolBase MergeTreeReadPoolParallelReplicas( ParallelReadingExtension extension_, RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -30,12 +31,13 @@ class MergeTreeReadPoolParallelReplicas : public MergeTreeReadPoolBase private: mutable std::mutex mutex; + LoggerPtr log = getLogger("MergeTreeReadPoolParallelReplicas"); const ParallelReadingExtension extension; const CoordinationMode coordination_mode; size_t min_marks_per_task{0}; + size_t mark_segment_size{0}; RangesInDataPartsDescription buffered_ranges; bool no_more_tasks_available{false}; - LoggerPtr log = getLogger("MergeTreeReadPoolParallelReplicas"); }; } diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp index 6b5cf9784232..2fdf65dcf36d 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp @@ -1,5 +1,10 @@ #include +namespace ProfileEvents +{ +extern const Event ParallelReplicasReadMarks; +} + namespace DB { @@ -12,6 +17,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd ParallelReadingExtension extension_, CoordinationMode mode_, RangesInDataParts parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -22,6 +28,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, @@ -43,11 +50,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd for (const auto & part : parts_ranges) buffered_tasks.push_back({part.data_part->info, MarkRanges{}}); - extension.all_callback(InitialAllRangesAnnouncement( - mode, - parts_ranges.getDescriptions(), - extension.number_of_current_replica - )); + extension.sendInitialRequest(mode, parts_ranges, /*mark_segment_size_=*/0); } MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicasInOrder::getTask(size_t task_idx, MergeTreeReadTask * previous_task) @@ -68,20 +71,20 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicasInOrder::getTask(size_t ta { auto result = std::move(desc.ranges); desc.ranges = MarkRanges{}; + ProfileEvents::increment(ProfileEvents::ParallelReplicasReadMarks, desc.ranges.getNumberOfMarks()); return result; } } return std::nullopt; }; - if (auto result = get_from_buffer(); result) + if (auto result = get_from_buffer()) return createTask(per_part_infos[task_idx], std::move(*result), previous_task); if (no_more_tasks) return nullptr; - auto response - = extension.callback(ParallelReadRequest(mode, extension.number_of_current_replica, min_marks_per_task * request.size(), request)); + auto response = extension.sendReadRequest(mode, min_marks_per_task * request.size(), request); if (!response || response->description.empty() || response->finish) { @@ -97,7 +100,7 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicasInOrder::getTask(size_t ta std::move(new_ranges.begin(), new_ranges.end(), std::back_inserter(old_ranges)); } - if (auto result = get_from_buffer(); result) + if (auto result = get_from_buffer()) return createTask(per_part_infos[task_idx], std::move(*result), previous_task); return nullptr; diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h index 22841bea212d..98a4d95768ad 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h @@ -12,6 +12,7 @@ class MergeTreeReadPoolParallelReplicasInOrder : public MergeTreeReadPoolBase ParallelReadingExtension extension_, CoordinationMode mode_, RangesInDataParts parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 1a0709faf1cd..a7ed91c8fc1e 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -15,6 +15,39 @@ #include #include +namespace +{ + +template +struct TelemetryWrapper +{ + TelemetryWrapper(Func callback_, ProfileEvents::Event event_, std::string span_name_) + : callback(std::move(callback_)), event(event_), span_name(std::move(span_name_)) + { + } + + template + auto operator()(Args &&... args) + { + DB::OpenTelemetry::SpanHolder span(span_name); + DB::ProfileEventTimeIncrement increment(event); + return callback(std::forward(args)...); + } + +private: + Func callback; + ProfileEvents::Event event; + std::string span_name; +}; + +} + +namespace ProfileEvents +{ +extern const Event ParallelReplicasAnnouncementMicroseconds; +extern const Event ParallelReplicasReadRequestMicroseconds; +} + namespace DB { @@ -23,6 +56,31 @@ namespace ErrorCodes extern const int QUERY_WAS_CANCELLED; } +ParallelReadingExtension::ParallelReadingExtension( + MergeTreeAllRangesCallback all_callback_, + MergeTreeReadTaskCallback callback_, + size_t number_of_current_replica_, + size_t total_nodes_count_) + : number_of_current_replica(number_of_current_replica_), total_nodes_count(total_nodes_count_) +{ + all_callback = TelemetryWrapper{ + std::move(all_callback_), ProfileEvents::ParallelReplicasAnnouncementMicroseconds, "ParallelReplicasAnnouncement"}; + + callback = TelemetryWrapper{ + std::move(callback_), ProfileEvents::ParallelReplicasReadRequestMicroseconds, "ParallelReplicasReadRequest"}; +} + +void ParallelReadingExtension::sendInitialRequest(CoordinationMode mode, const RangesInDataParts & ranges, size_t mark_segment_size) const +{ + all_callback(InitialAllRangesAnnouncement{mode, ranges.getDescriptions(), number_of_current_replica, mark_segment_size}); +} + +std::optional ParallelReadingExtension::sendReadRequest( + CoordinationMode mode, size_t min_number_of_marks, const RangesInDataPartsDescription & description) const +{ + return callback(ParallelReadRequest{mode, number_of_current_replica, min_number_of_marks, description}); +} + MergeTreeSelectProcessor::MergeTreeSelectProcessor( MergeTreeReadPoolPtr pool_, MergeTreeSelectAlgorithmPtr algorithm_, diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 7a9cebbcb2ed..8a9e3580a9f7 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -1,9 +1,12 @@ #pragma once + +#include #include -#include #include -#include #include +#include +#include + #include @@ -22,11 +25,27 @@ struct ChunkAndProgress bool is_finished = false; }; -struct ParallelReadingExtension +class ParallelReadingExtension { +public: + ParallelReadingExtension( + MergeTreeAllRangesCallback all_callback_, + MergeTreeReadTaskCallback callback_, + size_t number_of_current_replica_, + size_t total_nodes_count_); + + void sendInitialRequest(CoordinationMode mode, const RangesInDataParts & ranges, size_t mark_segment_size) const; + + std::optional + sendReadRequest(CoordinationMode mode, size_t min_number_of_marks, const RangesInDataPartsDescription & description) const; + + size_t getTotalNodesCount() const { return total_nodes_count; } + +private: MergeTreeAllRangesCallback all_callback; MergeTreeReadTaskCallback callback; - size_t number_of_current_replica{0}; + const size_t number_of_current_replica; + const size_t total_nodes_count; }; /// Base class for MergeTreeThreadSelectAlgorithm and MergeTreeSelectAlgorithm diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index f06dbabaeb5d..3b33bfce2676 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -34,6 +34,7 @@ class MergeTreeSequentialSource : public ISource const MergeTreeData & storage_, const StorageSnapshotPtr & storage_snapshot_, MergeTreeData::DataPartPtr data_part_, + AlterConversionsPtr alter_conversions_, Names columns_to_read_, std::optional mark_ranges_, bool apply_deleted_mask, @@ -58,6 +59,9 @@ class MergeTreeSequentialSource : public ISource /// Data part will not be removed if the pointer owns it MergeTreeData::DataPartPtr data_part; + /// Alter and mutation commands that are required to be applied to the part on-fly. + AlterConversionsPtr alter_conversions; + /// Columns we have to read (each Block from read will contain them) Names columns_to_read; @@ -87,6 +91,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( const MergeTreeData & storage_, const StorageSnapshotPtr & storage_snapshot_, MergeTreeData::DataPartPtr data_part_, + AlterConversionsPtr alter_conversions_, Names columns_to_read_, std::optional mark_ranges_, bool apply_deleted_mask, @@ -96,6 +101,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( , storage(storage_) , storage_snapshot(storage_snapshot_) , data_part(std::move(data_part_)) + , alter_conversions(std::move(alter_conversions_)) , columns_to_read(std::move(columns_to_read_)) , read_with_direct_io(read_with_direct_io_) , mark_ranges(std::move(mark_ranges_)) @@ -109,8 +115,6 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( LOG_DEBUG(log, "Reading {} marks from part {}, total {} rows starting from the beginning of the part", data_part->getMarksCount(), data_part->name, data_part->rows_count); - auto alter_conversions = storage.getAlterConversionsForPart(data_part); - /// Note, that we don't check setting collaborate_with_coordinator presence, because this source /// is only used in background merges. addTotalRowsApprox(data_part->rows_count); @@ -299,6 +303,7 @@ Pipe createMergeTreeSequentialSource( const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, + AlterConversionsPtr alter_conversions, Names columns_to_read, std::optional mark_ranges, std::shared_ptr> filtered_rows_count, @@ -315,7 +320,8 @@ Pipe createMergeTreeSequentialSource( columns_to_read.emplace_back(RowExistsColumn::name); auto column_part_source = std::make_shared(type, - storage, storage_snapshot, data_part, columns_to_read, std::move(mark_ranges), + storage, storage_snapshot, data_part, alter_conversions, + columns_to_read, std::move(mark_ranges), /*apply_deleted_mask=*/ false, read_with_direct_io, prefetch); Pipe pipe(std::move(column_part_source)); @@ -346,6 +352,7 @@ class ReadFromPart final : public ISourceStep const MergeTreeData & storage_, const StorageSnapshotPtr & storage_snapshot_, MergeTreeData::DataPartPtr data_part_, + AlterConversionsPtr alter_conversions_, Names columns_to_read_, bool apply_deleted_mask_, std::optional filter_, @@ -356,6 +363,7 @@ class ReadFromPart final : public ISourceStep , storage(storage_) , storage_snapshot(storage_snapshot_) , data_part(std::move(data_part_)) + , alter_conversions(std::move(alter_conversions_)) , columns_to_read(std::move(columns_to_read_)) , apply_deleted_mask(apply_deleted_mask_) , filter(std::move(filter_)) @@ -399,6 +407,7 @@ class ReadFromPart final : public ISourceStep storage, storage_snapshot, data_part, + alter_conversions, columns_to_read, std::move(mark_ranges), /*filtered_rows_count=*/ nullptr, @@ -414,6 +423,7 @@ class ReadFromPart final : public ISourceStep const MergeTreeData & storage; StorageSnapshotPtr storage_snapshot; MergeTreeData::DataPartPtr data_part; + AlterConversionsPtr alter_conversions; Names columns_to_read; bool apply_deleted_mask; std::optional filter; @@ -427,6 +437,7 @@ void createReadFromPartStep( const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, + AlterConversionsPtr alter_conversions, Names columns_to_read, bool apply_deleted_mask, std::optional filter, @@ -434,7 +445,8 @@ void createReadFromPartStep( LoggerPtr log) { auto reading = std::make_unique(type, - storage, storage_snapshot, std::move(data_part), + storage, storage_snapshot, + std::move(data_part), std::move(alter_conversions), std::move(columns_to_read), apply_deleted_mask, std::move(filter), std::move(context), log); diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index 1b05512b9a30..5b7c80385f67 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -21,6 +21,7 @@ Pipe createMergeTreeSequentialSource( const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, + AlterConversionsPtr alter_conversions, Names columns_to_read, std::optional mark_ranges, std::shared_ptr> filtered_rows_count, @@ -36,6 +37,7 @@ void createReadFromPartStep( const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, + AlterConversionsPtr alter_conversions, Names columns_to_read, bool apply_deleted_mask, std::optional filter, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e1ca6347d3b2..1e877362c152 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -114,6 +114,7 @@ static UInt64 getExistingRowsCount(const Block & block) static void splitAndModifyMutationCommands( MergeTreeData::DataPartPtr part, StorageMetadataPtr metadata_snapshot, + AlterConversionsPtr alter_conversions, const MutationCommands & commands, MutationCommands & for_interpreter, MutationCommands & for_file_renames, @@ -179,8 +180,6 @@ static void splitAndModifyMutationCommands( } - auto alter_conversions = part->storage.getAlterConversionsForPart(part); - /// We don't add renames from commands, instead we take them from rename_map. /// It's important because required renames depend not only on part's data version (i.e. mutation version) /// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter @@ -296,7 +295,6 @@ static void splitAndModifyMutationCommands( } } - auto alter_conversions = part->storage.getAlterConversionsForPart(part); /// We don't add renames from commands, instead we take them from rename_map. /// It's important because required renames depend not only on part's data version (i.e. mutation version) /// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter @@ -2160,6 +2158,15 @@ bool MutateTask::prepare() ctx->num_mutations = std::make_unique(CurrentMetrics::PartMutation); + MergeTreeData::IMutationsSnapshot::Params params + { + .metadata_version = ctx->metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = ctx->source_part->getMetadataVersion(), + }; + + auto mutations_snapshot = ctx->data->getMutationsSnapshot(params); + auto alter_conversions = MergeTreeData::getAlterConversionsForPart(ctx->source_part, mutations_snapshot, ctx->metadata_snapshot, ctx->context); + auto context_for_reading = Context::createCopy(ctx->context); /// Allow mutations to work when force_index_by_date or force_primary_key is on. @@ -2174,7 +2181,7 @@ bool MutateTask::prepare() ctx->commands_for_part.emplace_back(command); if (ctx->source_part->isStoredOnDisk() && !isStorageTouchedByMutations( - ctx->source_part, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading)) + ctx->source_part, mutations_snapshot, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading)) { NameSet files_to_copy_instead_of_hardlinks; auto settings_ptr = ctx->data->getSettings(); @@ -2234,8 +2241,13 @@ bool MutateTask::prepare() context_for_reading->setSetting("read_from_filesystem_cache_if_exists_otherwise_bypass_cache", 1); MutationHelpers::splitAndModifyMutationCommands( - ctx->source_part, ctx->metadata_snapshot, - ctx->commands_for_part, ctx->for_interpreter, ctx->for_file_renames, ctx->log); + ctx->source_part, + ctx->metadata_snapshot, + alter_conversions, + ctx->commands_for_part, + ctx->for_interpreter, + ctx->for_file_renames, + ctx->log); ctx->stage_progress = std::make_unique(1.0); @@ -2247,7 +2259,8 @@ bool MutateTask::prepare() settings.apply_deleted_mask = false; ctx->interpreter = std::make_unique( - *ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->for_interpreter, + *ctx->data, ctx->source_part, alter_conversions, + ctx->metadata_snapshot, ctx->for_interpreter, ctx->metadata_snapshot->getColumns().getNamesOfPhysical(), context_for_reading, settings); ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices(); diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index f46b4de10b7d..4067480cd7fd 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -185,7 +185,8 @@ class ParallelReplicasReadingCoordinator::ImplInterface void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { if (++sent_initial_requests > replicas_count) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Initiator received more initial requests than there are replicas"); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Initiator received more initial requests than there are replicas: replica_num={}", announcement.replica_num); doHandleInitialAllRangesAnnouncement(std::move(announcement)); } @@ -211,14 +212,11 @@ using PartRefs = std::deque; class DefaultCoordinator : public ParallelReplicasReadingCoordinator::ImplInterface { public: - explicit DefaultCoordinator(size_t replicas_count_, size_t mark_segment_size_) + explicit DefaultCoordinator(size_t replicas_count_) : ParallelReplicasReadingCoordinator::ImplInterface(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; @@ -231,7 +229,7 @@ class DefaultCoordinator : public ParallelReplicasReadingCoordinator::ImplInterf 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 mark_segment_size{0}; bool state_initialized{false}; size_t finished_replicas{0}; @@ -393,7 +391,11 @@ void DefaultCoordinator::initializeReadingState(InitialAllRangesAnnouncement ann 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, "; ")); + mark_segment_size = announcement.mark_segment_size; + if (mark_segment_size == 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Zero value provided for `mark_segment_size`"); + + LOG_DEBUG(log, "Reading state is fully initialized: {}, mark_segment_size: {}", fmt::join(all_parts_to_read, "; "), mark_segment_size); } void DefaultCoordinator::markReplicaAsUnavailable(size_t replica_number) @@ -433,9 +435,9 @@ void DefaultCoordinator::setProgressCallback() void DefaultCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { - const auto replica_num = announcement.replica_num; + LOG_DEBUG(log, "Initial request: {}", announcement.describe()); - LOG_DEBUG(log, "Initial request from replica {}: {}", announcement.replica_num, announcement.describe()); + const auto replica_num = announcement.replica_num; initializeReadingState(std::move(announcement)); @@ -839,6 +841,7 @@ class InOrderCoordinator : public ParallelReplicasReadingCoordinator::ImplInterf Parts all_parts_to_read; size_t total_rows_to_read = 0; + bool state_initialized{false}; LoggerPtr log = getLogger(fmt::format("{}{}", magic_enum::enum_name(mode), "Coordinator")); }; @@ -858,7 +861,9 @@ void InOrderCoordinator::markReplicaAsUnavailable(size_t replica_number) template void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { - LOG_TRACE(log, "Received an announcement {}", announcement.describe()); + LOG_TRACE(log, "Received an announcement : {}", announcement.describe()); + + ++stats[announcement.replica_num].number_of_requests; size_t new_rows_to_read = 0; @@ -868,13 +873,16 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), [&part] (const Part & other) { return other.description.info == part.info; }); - /// We have the same part - add the info about presence on current replica to it + /// We have the same part - add the info about presence on the corresponding replica to it if (the_same_it != all_parts_to_read.end()) { the_same_it->replicas.insert(announcement.replica_num); continue; } + if (state_initialized) + continue; + auto covering_or_the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), [&part] (const Part & other) { return other.description.info.contains(part.info) || part.info.contains(other.description.info); }); @@ -889,9 +897,10 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa std::sort(ranges.begin(), ranges.end()); } - ++stats[announcement.replica_num].number_of_requests; + state_initialized = true; - if (new_rows_to_read > 0) + // progress_callback is not set when local plan is used for initiator + if (progress_callback && new_rows_to_read > 0) { Progress progress; progress.total_rows_to_read = new_rows_to_read; @@ -911,7 +920,7 @@ ParallelReadResponse InOrderCoordinator::handleRequest(ParallelReadRequest "Replica {} decided to read in {} mode, not in {}. This is a bug", request.replica_num, magic_enum::enum_name(request.mode), magic_enum::enum_name(mode)); - LOG_TRACE(log, "Got request from replica {}, data {}", request.replica_num, request.describe()); + LOG_TRACE(log, "Got read request: {}", request.describe()); ParallelReadResponse response; response.description = request.description; @@ -925,8 +934,15 @@ ParallelReadResponse InOrderCoordinator::handleRequest(ParallelReadRequest if (global_part_it == all_parts_to_read.end()) continue; + if (global_part_it->replicas.empty()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Part {} requested by replica {} is not registered in working set", + part.info.getPartNameV1(), + request.replica_num); + if (!global_part_it->replicas.contains(request.replica_num)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} doesn't exist on replica {} according to the global state", part.info.getPartNameV1(), request.replica_num); + continue; size_t current_mark_size = 0; @@ -1043,7 +1059,7 @@ void ParallelReplicasReadingCoordinator::initialize(CoordinationMode mode) switch (mode) { case CoordinationMode::Default: - pimpl = std::make_unique(replicas_count, mark_segment_size); + pimpl = std::make_unique(replicas_count); break; case CoordinationMode::WithOrder: pimpl = std::make_unique>(replicas_count); @@ -1053,6 +1069,7 @@ void ParallelReplicasReadingCoordinator::initialize(CoordinationMode mode) break; } + // progress_callback is not set when local plan is used for initiator if (progress_callback) pimpl->setProgressCallback(std::move(progress_callback)); @@ -1060,8 +1077,7 @@ void ParallelReplicasReadingCoordinator::initialize(CoordinationMode mode) pimpl->markReplicaAsUnavailable(replica); } -ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator(size_t replicas_count_, size_t mark_segment_size_) - : replicas_count(replicas_count_), mark_segment_size(mark_segment_size_) +ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator(size_t replicas_count_) : replicas_count(replicas_count_) { } diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h index 8b463fda395b..ad51d20f5531 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_, size_t mark_segment_size_ = 0); + explicit ParallelReplicasReadingCoordinator(size_t replicas_count_); ~ParallelReplicasReadingCoordinator(); void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement); @@ -35,7 +35,6 @@ class ParallelReplicasReadingCoordinator std::mutex mutex; const size_t replicas_count{0}; - size_t mark_segment_size{0}; std::unique_ptr pimpl; ProgressCallback progress_callback; // store the callback only to bypass it to coordinator implementation std::set replicas_used; diff --git a/src/Storages/MergeTree/RangesInDataPart.h b/src/Storages/MergeTree/RangesInDataPart.h index bf9e4c7dfb2e..966637d08124 100644 --- a/src/Storages/MergeTree/RangesInDataPart.h +++ b/src/Storages/MergeTree/RangesInDataPart.h @@ -42,7 +42,6 @@ struct RangesInDataPartsDescription: public std::dequeznode_name, MutationStatus(entry, format_version)) - .first->second; + auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)).first->second; + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands); - updateAlterConversionsMutations(entry->commands, alter_conversions_mutations, /* remove= */ false); NOEXCEPT_SCOPE({ for (const auto & pair : entry->block_numbers) { @@ -1076,7 +1075,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation( } mutations_by_znode.erase(it); - /// updateAlterConversionsMutations() will be called in updateMutations() + /// decrementMutationsCounters() will be called in updateMutations() LOG_DEBUG(log, "Removed mutation {} from local state.", entry->znode_name); } @@ -1901,50 +1900,47 @@ ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue::getMergePredicate(zk } -MutationCommands ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const +MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const { - int32_t part_metadata_version = part->getMetadataVersion(); - int32_t metadata_version = storage.getInMemoryMetadataPtr()->getMetadataVersion(); - - chassert(alter_conversions_mutations >= 0); - /// NOTE: that just checking part_metadata_version is not enough, since we - /// need to check for non-metadata mutations as well. - if (alter_conversions_mutations == 0 && metadata_version == part_metadata_version) - return {}; - - std::unique_lock lock(state_mutex); - auto in_partition = mutations_by_partition.find(part->info.partition_id); if (in_partition == mutations_by_partition.end()) return {}; Int64 part_data_version = part->info.getDataVersion(); + Int64 part_metadata_version = part->getMetadataVersion(); + MutationCommands result; - bool seen_all_data_mutations = false; - bool seen_all_metadata_mutations = false; + bool seen_all_data_mutations = !hasDataMutations(); + bool seen_all_metadata_mutations = part_metadata_version >= params.metadata_version; + + if (seen_all_data_mutations && seen_all_metadata_mutations) + return {}; auto add_to_result = [&](const ReplicatedMergeTreeMutationEntryPtr & entry) { for (const auto & command : entry->commands | std::views::reverse) - if (AlterConversions::supportsMutationCommandType(command.type)) - result.emplace_back(command); + { + if (AlterConversions::isSupportedMetadataMutation(command.type)) + result.push_back(command); + else if (params.need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + result.push_back(command); + } }; /// Here we return mutation commands for part which has bigger alter version than part metadata version. /// Please note, we don't use getDataVersion(). It's because these alter commands are used for in-fly conversions /// of part's metadata. - for (const auto & [mutation_version, mutation_status] : in_partition->second | std::views::reverse) + for (const auto & [mutation_version, entry] : in_partition->second | std::views::reverse) { if (seen_all_data_mutations && seen_all_metadata_mutations) break; - auto & entry = mutation_status->entry; - auto alter_version = entry->alter_version; + if (alter_version != -1) { - if (alter_version > metadata_version) + if (seen_all_metadata_mutations || alter_version > params.metadata_version) continue; /// We take commands with bigger metadata version @@ -1953,7 +1949,7 @@ MutationCommands ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const else seen_all_metadata_mutations = true; } - else + else if (!seen_all_data_mutations) { if (mutation_version > part_data_version) add_to_result(entry); @@ -1965,6 +1961,104 @@ MutationCommands ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const return result; } +NameSet ReplicatedMergeTreeQueue::MutationsSnapshot::getAllUpdatedColumns() const +{ + if (!hasDataMutations()) + return {}; + + NameSet res; + for (const auto & [partition_id, mutations] : mutations_by_partition) + { + for (const auto & [version, entry] : mutations) + { + auto names = entry->commands.getAllUpdatedColumns(); + std::move(names.begin(), names.end(), std::inserter(res, res.end())); + } + } + return res; +} + +MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapshot(const MutationsSnapshot::Params & params) const +{ + std::lock_guard lock(state_mutex); + + MutationsSnapshot::Info info + { + .num_data_mutations = num_data_mutations_to_apply, + .num_metadata_mutations = num_metadata_mutations_to_apply, + }; + + auto res = std::make_shared(params, std::move(info)); + + bool need_data_mutations = res->hasDataMutations(); + bool need_metatadata_mutations = params.min_part_metadata_version < params.metadata_version; + + if (!need_data_mutations && !need_metatadata_mutations) + return res; + + auto is_supported_command = [&](const auto & command) + { + if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + return true; + + if (need_metatadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) + return true; + + return false; + }; + + for (const auto & [partition_id, mutations] : mutations_by_partition) + { + auto & in_partition = res->mutations_by_partition[partition_id]; + + bool seen_all_data_mutations = !need_data_mutations; + bool seen_all_metadata_mutations = !need_metatadata_mutations; + + for (const auto & [mutation_version, status] : mutations | std::views::reverse) + { + if (seen_all_data_mutations && seen_all_metadata_mutations) + break; + + auto alter_version = status->entry->alter_version; + + if (alter_version != -1) + { + if (seen_all_metadata_mutations || alter_version > params.metadata_version) + continue; + + /// We take commands with bigger metadata version + if (alter_version > params.min_part_metadata_version) + { + /// Copy a pointer to the whole entry to avoid extracting and copying commands. + /// Required commands will be copied later only for specific parts. + if (std::ranges::any_of(status->entry->commands, is_supported_command)) + in_partition.emplace(mutation_version, status->entry); + } + else + { + seen_all_metadata_mutations = true; + } + } + else if (!seen_all_data_mutations) + { + if (!status->is_done) + { + /// Copy a pointer to the whole entry to avoid extracting and copying commands. + /// Required commands will be copied later only for specific parts. + if (std::ranges::any_of(status->entry->commands, is_supported_command)) + in_partition.emplace(mutation_version, status->entry); + } + else + { + seen_all_data_mutations = true; + } + } + } + } + + return res; +} + MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version, Strings & mutation_ids) const { @@ -2045,7 +2139,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep mutation.parts_to_do.clear(); } - updateAlterConversionsMutations(mutation.entry->commands, alter_conversions_mutations, /* remove= */ true); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, mutation.entry->commands); } else if (mutation.parts_to_do.size() == 0) { @@ -2102,7 +2196,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep LOG_TRACE(log, "Finishing data alter with version {} for entry {}", entry->alter_version, entry->znode_name); alter_sequence.finishDataAlter(entry->alter_version, lock); } - updateAlterConversionsMutations(entry->commands, alter_conversions_mutations, /* remove= */ true); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands); } } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index e300078a35af..6ec8818b0c65 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -151,8 +152,11 @@ class ReplicatedMergeTreeQueue /// Mapping from znode path to Mutations Status std::map mutations_by_znode; - /// Unfinished mutations that is required AlterConversions (see getAlterMutationCommandsForPart()) - std::atomic alter_conversions_mutations = 0; + + /// Unfinished mutations that are required for AlterConversions. + Int64 num_data_mutations_to_apply = 0; + Int64 num_metadata_mutations_to_apply = 0; + /// Partition -> (block_number -> MutationStatus) std::unordered_map> mutations_by_partition; /// Znode ID of the latest mutation that is done. @@ -410,10 +414,26 @@ class ReplicatedMergeTreeQueue MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version, Strings & mutation_ids) const; + struct MutationsSnapshot : public MergeTreeData::IMutationsSnapshot + { + public: + MutationsSnapshot() = default; + MutationsSnapshot(Params params_, Info info_) : IMutationsSnapshot(std::move(params_), std::move(info_)) {} + + using Params = MergeTreeData::IMutationsSnapshot::Params; + using MutationsByPartititon = std::unordered_map>; + + MutationsByPartititon mutations_by_partition; + + MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override; + std::shared_ptr cloneEmpty() const override { return std::make_shared(); } + NameSet getAllUpdatedColumns() const override; + }; + /// Return mutation commands for part which could be not applied to /// it according to part mutation version. Used when we apply alter commands on fly, /// without actual data modification on disk. - MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const; + MergeTreeData::MutationsSnapshotPtr getMutationsSnapshot(const MutationsSnapshot::Params & params) const; /// Mark finished mutations as done. If the function needs to be called again at some later time /// (because some mutations are probably done but we are not sure yet), returns true. diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index 2ce0e20dcd20..f6859b269082 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -2,10 +2,10 @@ #include #include -#include +#include #include #include -#include +#include #include @@ -14,25 +14,29 @@ namespace DB namespace ErrorCodes { - extern const int UNKNOWN_PROTOCOL; - extern const int UNKNOWN_ELEMENT_OF_ENUM; +extern const int UNKNOWN_PROTOCOL; +extern const int UNKNOWN_ELEMENT_OF_ENUM; } namespace { - CoordinationMode validateAndGet(uint8_t candidate) - { - if (candidate <= static_cast(CoordinationMode::MAX)) - return static_cast(candidate); +CoordinationMode validateAndGet(uint8_t candidate) +{ + if (candidate <= static_cast(CoordinationMode::MAX)) + return static_cast(candidate); - throw Exception(ErrorCodes::UNKNOWN_ELEMENT_OF_ENUM, "Unknown reading mode: {}", candidate); - } + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_OF_ENUM, "Unknown reading mode: {}", candidate); +} } -void ParallelReadRequest::serialize(WriteBuffer & out) const +void ParallelReadRequest::serialize(WriteBuffer & out, UInt64 initiator_protocol_version) const { - UInt64 version = DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION; - /// Must be the first + /// Previously we didn't maintain backward compatibility and every change was breaking. + /// Particularly, we had an equality check for the version. To work around that code + /// in previous server versions we now have to lie to them about the version. + const UInt64 version = initiator_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL + ? DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION + : DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION; writeIntBinary(version, out); writeIntBinary(mode, out); @@ -44,9 +48,7 @@ void ParallelReadRequest::serialize(WriteBuffer & out) const String ParallelReadRequest::describe() const { - String result; - result += fmt::format("replica_num: {} \n", replica_num); - result += fmt::format("min_num_of_marks: {} \n", min_number_of_marks); + String result = fmt::format("replica_num {}, min_num_of_marks {}, ", replica_num, min_number_of_marks); result += description.describe(); return result; } @@ -55,10 +57,12 @@ ParallelReadRequest ParallelReadRequest::deserialize(ReadBuffer & in) { UInt64 version; readIntBinary(version, in); - if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION) - throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading "\ - "from replicas differ. Got: {}, supported version: {}", - version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); + if (version < DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION) + throw Exception( + ErrorCodes::UNKNOWN_PROTOCOL, + "Parallel replicas protocol version is too old. Got: {}, min supported version: {}", + version, + DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION); CoordinationMode mode; size_t replica_num; @@ -72,12 +76,7 @@ ParallelReadRequest ParallelReadRequest::deserialize(ReadBuffer & in) readIntBinary(min_number_of_marks, in); description.deserialize(in); - return ParallelReadRequest( - mode, - replica_num, - min_number_of_marks, - std::move(description) - ); + return ParallelReadRequest(mode, replica_num, min_number_of_marks, std::move(description)); } void ParallelReadRequest::merge(ParallelReadRequest & other) @@ -88,9 +87,14 @@ void ParallelReadRequest::merge(ParallelReadRequest & other) description.merge(other.description); } -void ParallelReadResponse::serialize(WriteBuffer & out) const +void ParallelReadResponse::serialize(WriteBuffer & out, UInt64 replica_protocol_version) const { - UInt64 version = DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION; + /// Previously we didn't maintain backward compatibility and every change was breaking. + /// Particularly, we had an equality check for the version. To work around that code + /// in previous server versions we now have to lie to them about the version. + UInt64 version = replica_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL + ? DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION + : DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION; /// Must be the first writeIntBinary(version, out); @@ -107,44 +111,51 @@ void ParallelReadResponse::deserialize(ReadBuffer & in) { UInt64 version; readIntBinary(version, in); - if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION) - throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading " \ - "from replicas differ. Got: {}, supported version: {}", - version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); + if (version < DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION) + throw Exception( + ErrorCodes::UNKNOWN_PROTOCOL, + "Parallel replicas protocol version is too old. Got: {}, min supported version: {}", + version, + DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION); readBoolText(finish, in); description.deserialize(in); } -void InitialAllRangesAnnouncement::serialize(WriteBuffer & out) const +void InitialAllRangesAnnouncement::serialize(WriteBuffer & out, UInt64 initiator_protocol_version) const { - UInt64 version = DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION; - /// Must be the first + /// Previously we didn't maintain backward compatibility and every change was breaking. + /// Particularly, we had an equality check for the version. To work around that code + /// in previous server versions we now have to lie to them about the version. + UInt64 version = initiator_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL + ? DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION + : DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION; writeIntBinary(version, out); writeIntBinary(mode, out); description.serialize(out); writeIntBinary(replica_num, out); + if (initiator_protocol_version >= DBMS_PARALLEL_REPLICAS_MIN_VERSION_WITH_MARK_SEGMENT_SIZE_FIELD) + writeIntBinary(mark_segment_size, out); } String InitialAllRangesAnnouncement::describe() { - String result; - result += description.describe(); - result += fmt::format("----------\nReceived from {} replica\n", replica_num); - return result; + return fmt::format("replica {}, mode {}, {}", replica_num, mode, description.describe()); } -InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffer & in) +InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffer & in, UInt64 replica_protocol_version) { UInt64 version; readIntBinary(version, in); - if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION) - throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading " \ - "from replicas differ. Got: {}, supported version: {}", - version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); + if (version < DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION) + throw Exception( + ErrorCodes::UNKNOWN_PROTOCOL, + "Parallel replicas protocol version is too old. Got: {}, min supported version: {}", + version, + DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION); CoordinationMode mode; RangesInDataPartsDescription description; @@ -156,11 +167,11 @@ InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffe description.deserialize(in); readIntBinary(replica_num, in); - return InitialAllRangesAnnouncement { - mode, - description, - replica_num - }; + size_t mark_segment_size = 128; + if (replica_protocol_version >= DBMS_PARALLEL_REPLICAS_MIN_VERSION_WITH_MARK_SEGMENT_SIZE_FIELD) + readIntBinary(mark_segment_size, in); + + return InitialAllRangesAnnouncement{mode, description, replica_num, mark_segment_size}; } } diff --git a/src/Storages/MergeTree/RequestResponse.h b/src/Storages/MergeTree/RequestResponse.h index 5f5516a6804d..96b65c45bfae 100644 --- a/src/Storages/MergeTree/RequestResponse.h +++ b/src/Storages/MergeTree/RequestResponse.h @@ -63,7 +63,7 @@ struct ParallelReadRequest /// Contains only data part names without mark ranges. RangesInDataPartsDescription description; - void serialize(WriteBuffer & out) const; + void serialize(WriteBuffer & out, UInt64 initiator_protocol_version) const; String describe() const; static ParallelReadRequest deserialize(ReadBuffer & in); void merge(ParallelReadRequest & other); @@ -78,7 +78,7 @@ struct ParallelReadResponse bool finish{false}; RangesInDataPartsDescription description; - void serialize(WriteBuffer & out) const; + void serialize(WriteBuffer & out, UInt64 replica_protocol_version) const; String describe() const; void deserialize(ReadBuffer & in); }; @@ -93,21 +93,18 @@ struct InitialAllRangesAnnouncement /// No default constructor, you must initialize all fields at once. InitialAllRangesAnnouncement( - CoordinationMode mode_, - RangesInDataPartsDescription description_, - size_t replica_num_) - : mode(mode_) - , description(description_) - , replica_num(replica_num_) + CoordinationMode mode_, RangesInDataPartsDescription description_, size_t replica_num_, size_t mark_segment_size_) + : mode(mode_), description(std::move(description_)), replica_num(replica_num_), mark_segment_size(mark_segment_size_) {} CoordinationMode mode; RangesInDataPartsDescription description; size_t replica_num; + size_t mark_segment_size; - void serialize(WriteBuffer & out) const; + void serialize(WriteBuffer & out, UInt64 initiator_protocol_version) const; String describe(); - static InitialAllRangesAnnouncement deserialize(ReadBuffer & in); + static InitialAllRangesAnnouncement deserialize(ReadBuffer & i, UInt64 replica_protocol_version); }; diff --git a/src/Storages/StorageFromMergeTreeDataPart.cpp b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.cpp similarity index 98% rename from src/Storages/StorageFromMergeTreeDataPart.cpp rename to src/Storages/MergeTree/StorageFromMergeTreeDataPart.cpp index 481d9e669017..280783245a22 100644 --- a/src/Storages/StorageFromMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.cpp @@ -30,7 +30,7 @@ void StorageFromMergeTreeDataPart::read( { query_plan.addStep(MergeTreeDataSelectExecutor(storage).readFromParts( parts, - alter_conversions, + mutations_snapshot, column_names, storage_snapshot, query_info, diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 82ebbf099888..6825752acd7b 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -18,10 +18,12 @@ class StorageFromMergeTreeDataPart final : public IStorage { public: /// Used in part mutation. - explicit StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) + explicit StorageFromMergeTreeDataPart( + const MergeTreeData::DataPartPtr & part_, + const MergeTreeData::MutationsSnapshotPtr & mutations_snapshot_) : IStorage(getIDFromPart(part_)) , parts({part_}) - , alter_conversions({part_->storage.getAlterConversionsForPart(part_)}) + , mutations_snapshot(mutations_snapshot_) , storage(part_->storage) , partition_id(part_->info.partition_id) { @@ -81,7 +83,7 @@ class StorageFromMergeTreeDataPart final : public IStorage private: const MergeTreeData::DataPartsVector parts; - const std::vector alter_conversions; + const MergeTreeData::MutationsSnapshotPtr mutations_snapshot; const MergeTreeData & storage; const String partition_id; const ReadFromMergeTree::AnalysisResultPtr analysis_result_ptr; diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index f736c863eeed..1aa9f5e23f8f 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -268,4 +268,13 @@ bool MutationCommands::containBarrierCommand() const return false; } +NameSet MutationCommands::getAllUpdatedColumns() const +{ + NameSet res; + for (const auto & command : *this) + for (const auto & [column_name, _] : command.column_to_update_expression) + res.insert(column_name); + return res; +} + } diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index f999aab1f4dc..5ae537bb657e 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -92,6 +92,7 @@ class MutationCommands : public std::vector /// stick with other commands. Commands from one set have already been validated /// to be executed without issues on the creation state. bool containBarrierCommand() const; + NameSet getAllUpdatedColumns() const; }; using MutationCommandsConstPtr = std::shared_ptr; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 0827321e296c..f40aa8ae4e8e 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1625,9 +1625,11 @@ std::tuple StorageMerge::evaluateDatabaseName(cons return {false, ast}; } -bool StorageMerge::supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr ctx) const +bool StorageMerge::supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr ctx) const { - return getFirstTable([&](const auto & table) { return !table->supportsTrivialCountOptimization(storage_snapshot, ctx); }) == nullptr; + /// Here we actually need storage snapshot of all nested tables. + /// But to avoid complexity pass nullptr to make more lightweight check in MergeTreeData. + return getFirstTable([&](const auto & table) { return !table->supportsTrivialCountOptimization(nullptr, ctx); }) == nullptr; } std::optional StorageMerge::totalRows(const Settings & settings) const diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 78dbb72c199c..e2a279361a3b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -512,21 +512,14 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context if (txn) txn->addMutation(shared_from_this(), mutation_id); - bool alter_conversions_mutations_updated = updateAlterConversionsMutations(entry.commands, alter_conversions_mutations, /* remove= */ false); - { std::lock_guard lock(currently_processing_in_background_mutex); - bool inserted = current_mutations_by_version.try_emplace(version, std::move(entry)).second; + auto [it, inserted] = current_mutations_by_version.try_emplace(version, std::move(entry)); if (!inserted) - { - if (alter_conversions_mutations_updated) - { - --alter_conversions_mutations; - chassert(alter_conversions_mutations >= 0); - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); - } + + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands); } LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); @@ -563,7 +556,7 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re if (static_cast(result_part->part_info.mutation) == it->first) mutation_backoff_policy.removePartFromFailed(failed_part->name); - updateAlterConversionsMutations(it->second.commands, alter_conversions_mutations, /* remove= */ true); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry.commands); } } else @@ -763,17 +756,15 @@ std::map StorageMergeTree::getUnfinishedMutationC std::map result; - for (const auto & kv : current_mutations_by_version) + for (const auto & [mutation_version, entry] : current_mutations_by_version) { - Int64 mutation_version = kv.first; - const MergeTreeMutationEntry & entry = kv.second; - const PartVersionWithName needle{mutation_version, ""}; + const PartVersionWithName needle{static_cast(mutation_version), ""}; auto versions_it = std::lower_bound( part_versions_with_names.begin(), part_versions_with_names.end(), needle, comparator); size_t parts_to_do = versions_it - part_versions_with_names.begin(); if (parts_to_do > 0) - result.emplace(entry.file_name, entry.commands); + result.emplace(entry.file_name, *entry.commands); } return result; } @@ -806,7 +797,7 @@ std::vector StorageMergeTree::getMutationsStatus() cons std::map block_numbers_map({{"", entry.block_number}}); - for (const MutationCommand & command : entry.commands) + for (const MutationCommand & command : *entry.commands) { WriteBufferFromOwnString buf; formatAST(*command.ast, buf, false, true); @@ -843,20 +834,15 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) auto it = current_mutations_by_version.find(mutation_version); if (it != current_mutations_by_version.end()) { - bool mutation_finished = true; if (std::optional min_version = getMinPartDataVersion()) - mutation_finished = *min_version > static_cast(mutation_version); - - to_kill.emplace(std::move(it->second)); - - if (!mutation_finished) { - const auto commands = it->second.commands; - current_mutations_by_version.erase(it); - updateAlterConversionsMutations(commands, alter_conversions_mutations, /* remove= */ true); + bool mutation_finished = *min_version > static_cast(mutation_version); + if (!mutation_finished) + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands); } - else - current_mutations_by_version.erase(it); + + to_kill.emplace(std::move(it->second)); + current_mutations_by_version.erase(it); } } @@ -904,6 +890,8 @@ void StorageMergeTree::loadDeduplicationLog() void StorageMergeTree::loadMutations() { + std::lock_guard lock(currently_processing_in_background_mutex); + for (const auto & disk : getDisks()) { for (auto it = disk->iterateDirectory(relative_data_path); it->isValid(); it->next()) @@ -912,7 +900,7 @@ void StorageMergeTree::loadMutations() { MergeTreeMutationEntry entry(disk, relative_data_path, it->name()); UInt64 block_number = entry.block_number; - LOG_DEBUG(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands.size()); + LOG_DEBUG(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands->size()); if (!entry.tid.isPrehistoric() && !entry.csn) { @@ -931,10 +919,11 @@ void StorageMergeTree::loadMutations() } } - auto inserted = current_mutations_by_version.try_emplace(block_number, std::move(entry)).second; + auto [entry_it, inserted] = current_mutations_by_version.try_emplace(block_number, std::move(entry)); if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); - updateAlterConversionsMutations(entry.commands, alter_conversions_mutations, /* remove= */ false); + + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry_it->second.commands); } else if (startsWith(it->name(), "tmp_mutation_")) { @@ -1282,7 +1271,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( size_t commands_size = 0; MutationCommands commands_for_size_validation; - for (const auto & command : it->second.commands) + for (const auto & command : *it->second.commands) { if (command.type != MutationCommand::Type::DROP_COLUMN && command.type != MutationCommand::Type::DROP_INDEX @@ -1326,11 +1315,11 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( const auto & single_mutation_commands = it->second.commands; - if (single_mutation_commands.containBarrierCommand()) + if (single_mutation_commands->containBarrierCommand()) { if (commands->empty()) { - commands->insert(commands->end(), single_mutation_commands.begin(), single_mutation_commands.end()); + commands->insert(commands->end(), single_mutation_commands->begin(), single_mutation_commands->end()); last_mutation_to_apply = it; } break; @@ -1338,7 +1327,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( else { current_ast_elements += commands_size; - commands->insert(commands->end(), single_mutation_commands.begin(), single_mutation_commands.end()); + commands->insert(commands->end(), single_mutation_commands->begin(), single_mutation_commands->end()); last_mutation_to_apply = it; } @@ -2455,32 +2444,80 @@ void StorageMergeTree::attachRestoredParts(MutableDataPartsVector && parts) } } +MutationCommands StorageMergeTree::MutationsSnapshot::getAlterMutationCommandsForPart(const DataPartPtr & part) const +{ + MutationCommands result; + UInt64 part_data_version = part->info.getDataVersion(); + + for (const auto & [mutation_version, commands] : mutations_by_version | std::views::reverse) + { + if (mutation_version <= part_data_version) + break; -MutationCommands StorageMergeTree::getAlterMutationCommandsForPart(const DataPartPtr & part) const + for (const auto & command : *commands | std::views::reverse) + { + if (params.need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + result.push_back(command); + else if (AlterConversions::isSupportedMetadataMutation(command.type)) + result.push_back(command); + } + } + + return result; +} + +NameSet StorageMergeTree::MutationsSnapshot::getAllUpdatedColumns() const { - /// NOTE: there is no need to check part metadata_version, since - /// ALTER_METADATA cannot be done asynchronously, like in - /// ReplicatedMergeTree. - chassert(alter_conversions_mutations >= 0); - if (alter_conversions_mutations == 0) + if (!hasDataMutations()) return {}; + NameSet res; + for (const auto & [version, commands] : mutations_by_version) + { + auto names = commands->getAllUpdatedColumns(); + std::move(names.begin(), names.end(), std::inserter(res, res.end())); + } + return res; +} + +MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const IMutationsSnapshot::Params & params) const +{ std::lock_guard lock(currently_processing_in_background_mutex); - UInt64 part_data_version = part->info.getDataVersion(); - MutationCommands result; + MutationsSnapshot::Info info + { + .num_data_mutations = num_data_mutations_to_apply, + .num_metadata_mutations = num_metadata_mutations_to_apply, + }; - for (const auto & [mutation_version, entry] : current_mutations_by_version | std::views::reverse) + auto res = std::make_shared(params, std::move(info)); + + bool need_data_mutations = res->hasDataMutations(); + bool need_metadata_mutations = num_metadata_mutations_to_apply > 0; + + if (!need_data_mutations && !need_metadata_mutations) + return res; + + for (const auto & [version, entry] : current_mutations_by_version) { - if (mutation_version <= part_data_version) - break; + bool has_required_command = std::ranges::any_of(*entry.commands, [&](const auto & command) + { + if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + return true; - for (const auto & command : entry.commands | std::views::reverse) - if (AlterConversions::supportsMutationCommandType(command.type)) - result.emplace_back(command); + if (need_metadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) + return true; + + return false; + }); + + /// Copy a pointer to all commands to avoid extracting and copying them. + /// Required commands will be copied later only for specific parts. + if (has_required_command) + res->mutations_by_version.emplace(version, entry.commands); } - return result; + return res; } void StorageMergeTree::startBackgroundMovesIfNeeded() diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 6223b5f50fae..ef333fe3f18b 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -147,8 +147,10 @@ class StorageMergeTree final : public MergeTreeData DataParts currently_merging_mutating_parts; std::map current_mutations_by_version; - /// Unfinished mutations that is required AlterConversions (see getAlterMutationCommandsForPart()) - std::atomic alter_conversions_mutations = 0; + + /// Unfinished mutations that are required for AlterConversions. + Int64 num_data_mutations_to_apply = 0; + Int64 num_metadata_mutations_to_apply = 0; std::atomic shutdown_called {false}; std::atomic flush_called {false}; @@ -308,9 +310,20 @@ class StorageMergeTree final : public MergeTreeData ContextPtr context; }; -protected: - /// Collect mutations that have to be applied on the fly: currently they are only RENAME COLUMN. - MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const override; + struct MutationsSnapshot : public IMutationsSnapshot + { + MutationsSnapshot() = default; + MutationsSnapshot(Params params_, Info info_) : IMutationsSnapshot(std::move(params_), std::move(info_)) {} + + using MutationsByVersion = std::map>; + MutationsByVersion mutations_by_version; + + MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override; + std::shared_ptr cloneEmpty() const override { return std::make_shared(); } + NameSet getAllUpdatedColumns() const override; + }; + + MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override; }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 14f2163a8d35..171e5b757d66 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5709,7 +5709,8 @@ std::optional StorageReplicatedMergeTree::distributedWriteFromClu { auto connection = std::make_shared( node.host_name, node.port, query_context->getGlobalContext()->getCurrentDatabase(), - node.user, node.password, SSHKey(), /*jwt*/"", node.quota_key, node.cluster, node.cluster_secret, + node.user, node.password, node.proto_send_chunked, node.proto_recv_chunked, + SSHKey(), /*jwt*/"", node.quota_key, node.cluster, node.cluster_secret, "ParallelInsertSelectInititiator", node.compression, node.secure @@ -9211,13 +9212,11 @@ bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const (!has_non_adaptive_index_granularity_parts && !other_replicas_fixed_granularity)); } - -MutationCommands StorageReplicatedMergeTree::getAlterMutationCommandsForPart(const DataPartPtr & part) const +MergeTreeData::MutationsSnapshotPtr StorageReplicatedMergeTree::getMutationsSnapshot(const IMutationsSnapshot::Params & params) const { - return queue.getAlterMutationCommandsForPart(part); + return queue.getMutationsSnapshot(params); } - void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() { if (areBackgroundMovesNeeded()) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 4beb5acf0c2f..a8900a4d1d73 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -934,7 +934,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData void waitMutationToFinishOnReplicas( const Strings & replicas, const String & mutation_id) const; - MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const override; + MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override; void startBackgroundMovesIfNeeded() override; diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 425552398dab..19679441d0d1 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -466,7 +466,7 @@ class CI: JobNames.STATELESS_TEST_FLAKY_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True, - timeout=3600, + timeout=3 * 3600, # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf # reference_job_name=JobNames.STATELESS_TEST_RELEASE, ), diff --git a/tests/ci/stress.py b/tests/ci/stress.py index a3cdd8b3d6f5..4e86f551b309 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -78,7 +78,7 @@ def get_options(i: int, upgrade_check: bool) -> str: client_options.append("ignore_drop_queries_probability=0.5") if random.random() < 0.2: - client_options.append("allow_experimental_parallel_reading_from_replicas=1") + client_options.append("enable_parallel_replicas=1") client_options.append("max_parallel_replicas=3") client_options.append("cluster_for_parallel_replicas='parallel_replicas'") client_options.append("parallel_replicas_for_non_replicated_merge_tree=1") diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a3d7e0e922de..665800f8b716 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -912,6 +912,7 @@ class SettingsRandomizer: "min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]), "max_parsing_threads": lambda: random.choice([0, 1, 10]), "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), + "parallel_replicas_local_plan": lambda: random.randint(0, 1), } @staticmethod diff --git a/tests/config/config.d/clusters.xml b/tests/config/config.d/clusters.xml index af26565d7e6a..ff42b5828e36 100644 --- a/tests/config/config.d/clusters.xml +++ b/tests/config/config.d/clusters.xml @@ -115,10 +115,6 @@ false - - 127.0.0.1 - 9000 - 127.0.0.2 9000 @@ -147,6 +143,10 @@ 127.0.0.8 9000 + + 127.0.0.1 + 9000 + 127.0.0.9 9000 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9a43dff48a89..d802a2166a3d 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4696,6 +4696,9 @@ def write_embedded_config(name, dest_dir, fix_log_level=False): if self.with_kerberized_hdfs: depends_on.append("kerberizedhdfs1") + if self.with_ldap: + depends_on.append("openldap") + if self.with_rabbitmq: depends_on.append("rabbitmq1") diff --git a/tests/integration/test_backward_compatibility/configs/clusters.xml b/tests/integration/test_backward_compatibility/configs/clusters.xml new file mode 100644 index 000000000000..ac773152df92 --- /dev/null +++ b/tests/integration/test_backward_compatibility/configs/clusters.xml @@ -0,0 +1,20 @@ + + + + + + node0 + 9000 + + + node1 + 9000 + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_backward_compatibility/test_parallel_replicas_protocol.py b/tests/integration/test_backward_compatibility/test_parallel_replicas_protocol.py new file mode 100644 index 000000000000..e1b9049ef5dc --- /dev/null +++ b/tests/integration/test_backward_compatibility/test_parallel_replicas_protocol.py @@ -0,0 +1,64 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) +cluster_name = "parallel_replicas" +nodes = [ + cluster.add_instance( + f"node{num}", + main_configs=["configs/clusters.xml"], + with_zookeeper=False, + image="clickhouse/clickhouse-server", + tag="23.11", # earlier versions lead to "Not found column sum(a) in block." exception 🤷 + stay_alive=True, + use_old_analyzer=True, + with_installed_binary=True, + ) + for num in range(2) +] + [ + cluster.add_instance( + "node2", + main_configs=["configs/clusters.xml"], + with_zookeeper=False, + use_old_analyzer=True, + ) +] + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_backward_compatability(start_cluster): + for node in nodes: + node.query("create table t (a UInt64) engine = MergeTree order by tuple()") + node.query("insert into t select number % 100000 from numbers_mt(1000000)") + + # all we want is the query to run without errors + for node in nodes: + assert ( + node.query( + """ + select sum(a) + from t + """, + settings={ + "cluster_for_parallel_replicas": "parallel_replicas", + "max_parallel_replicas": 3, + "allow_experimental_parallel_reading_from_replicas": 1, + "parallel_replicas_for_non_replicated_merge_tree": 1, + }, + ) + == "49999500000\n" + ) + + for node in nodes: + node.query("drop table t") diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 02ecf3c1367d..0d72f7c45b10 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -333,7 +333,7 @@ def test_receive_timeout2(started_cluster): # in packet receiving but there are replicas in process of # connection establishing. update_configs( - node_1_sleep_in_send_data=4000, + node_1_sleep_in_send_data=5000, node_2_sleep_in_send_tables_status=2000, node_3_sleep_in_send_tables_status=2000, ) diff --git a/tests/integration/test_ldap_external_user_directory/configs/remote_servers.xml b/tests/integration/test_ldap_external_user_directory/configs/remote_servers.xml new file mode 100644 index 000000000000..cf1bdf9dcb19 --- /dev/null +++ b/tests/integration/test_ldap_external_user_directory/configs/remote_servers.xml @@ -0,0 +1,18 @@ + + + + + + instance1 + 9000 + + + + + instance2 + 9000 + + + + + \ No newline at end of file diff --git a/tests/integration/test_ldap_external_user_directory/test.py b/tests/integration/test_ldap_external_user_directory/test.py index 39753794d633..0315c02b532e 100644 --- a/tests/integration/test_ldap_external_user_directory/test.py +++ b/tests/integration/test_ldap_external_user_directory/test.py @@ -7,8 +7,22 @@ LDAP_ADMIN_PASSWORD = "clickhouse" cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "instance", main_configs=["configs/ldap_with_role_mapping.xml"], with_ldap=True + +instance1 = cluster.add_instance( + "instance1", + main_configs=["configs/ldap_with_role_mapping.xml", "configs/remote_servers.xml"], + macros={"shard": 1, "replica": "instance1"}, + stay_alive=True, + with_ldap=True, + with_zookeeper=True, +) + +instance2 = cluster.add_instance( + "instance2", + main_configs=["configs/remote_servers.xml"], + macros={"shard": 1, "replica": "instance2"}, + stay_alive=True, + with_zookeeper=True, ) @@ -48,48 +62,143 @@ def add_ldap_group(ldap_cluster, group_cn, member_cn): assert code == 0 +def delete_ldap_group(ldap_cluster, group_cn): + code, (stdout, stderr) = ldap_cluster.ldap_container.exec_run( + [ + "sh", + "-c", + """ldapdelete -r 'cn={group_cn},dc=example,dc=org' \ +-H ldap://{host}:{port} -D "{admin_bind_dn}" -x -w {admin_password} + """.format( + host=ldap_cluster.ldap_host, + port=ldap_cluster.ldap_port, + admin_bind_dn=LDAP_ADMIN_BIND_DN, + admin_password=LDAP_ADMIN_PASSWORD, + group_cn=group_cn, + ), + ], + demux=True, + ) + logging.debug( + f"test_ldap_external_user_directory code:{code} stdout:{stdout}, stderr:{stderr}" + ) + assert code == 0 + + def test_authentication_pass(): - assert instance.query( - "select currentUser()", user="janedoe", password="qwerty" + assert instance1.query( + "SELECT currentUser()", user="janedoe", password="qwerty" ) == TSV([["janedoe"]]) def test_authentication_fail(): # User doesn't exist. - assert "doesnotexist: Authentication failed" in instance.query_and_get_error( + assert "doesnotexist: Authentication failed" in instance1.query_and_get_error( "SELECT currentUser()", user="doesnotexist" ) # Wrong password. - assert "janedoe: Authentication failed" in instance.query_and_get_error( + assert "janedoe: Authentication failed" in instance1.query_and_get_error( "SELECT currentUser()", user="janedoe", password="123" ) def test_role_mapping(ldap_cluster): - instance.query("CREATE ROLE role_1") - instance.query("CREATE ROLE role_2") + instance1.query("DROP ROLE IF EXISTS role_1") + instance1.query("DROP ROLE IF EXISTS role_2") + instance1.query("DROP ROLE IF EXISTS role_3") + instance1.query("CREATE ROLE role_1") + instance1.query("CREATE ROLE role_2") add_ldap_group(ldap_cluster, group_cn="clickhouse-role_1", member_cn="johndoe") add_ldap_group(ldap_cluster, group_cn="clickhouse-role_2", member_cn="johndoe") - assert instance.query( + assert instance1.query( "select currentUser()", user="johndoe", password="qwertz" ) == TSV([["johndoe"]]) - assert instance.query( + assert instance1.query( "select role_name from system.current_roles ORDER BY role_name", user="johndoe", password="qwertz", ) == TSV([["role_1"], ["role_2"]]) - instance.query("CREATE ROLE role_3") + instance1.query("CREATE ROLE role_3") add_ldap_group(ldap_cluster, group_cn="clickhouse-role_3", member_cn="johndoe") # Check that non-existing role in ClickHouse is ignored during role update # See https://github.com/ClickHouse/ClickHouse/issues/54318 add_ldap_group(ldap_cluster, group_cn="clickhouse-role_4", member_cn="johndoe") - assert instance.query( + assert instance1.query( "select role_name from system.current_roles ORDER BY role_name", user="johndoe", password="qwertz", ) == TSV([["role_1"], ["role_2"], ["role_3"]]) + + instance1.query("DROP ROLE role_1") + instance1.query("DROP ROLE role_2") + instance1.query("DROP ROLE role_3") + + delete_ldap_group(ldap_cluster, group_cn="clickhouse-role_1") + delete_ldap_group(ldap_cluster, group_cn="clickhouse-role_2") + delete_ldap_group(ldap_cluster, group_cn="clickhouse-role_3") + delete_ldap_group(ldap_cluster, group_cn="clickhouse-role_4") + + +def test_push_role_to_other_nodes(ldap_cluster): + instance1.query("DROP TABLE IF EXISTS distributed_table SYNC") + instance1.query("DROP TABLE IF EXISTS local_table SYNC") + instance2.query("DROP TABLE IF EXISTS local_table SYNC") + instance1.query("DROP ROLE IF EXISTS role_read") + + instance1.query("CREATE ROLE role_read") + instance1.query("GRANT SELECT ON *.* TO role_read") + + add_ldap_group(ldap_cluster, group_cn="clickhouse-role_read", member_cn="johndoe") + + assert instance1.query( + "select currentUser()", user="johndoe", password="qwertz" + ) == TSV([["johndoe"]]) + + instance1.query( + "CREATE TABLE IF NOT EXISTS local_table (id UInt32) ENGINE = MergeTree() ORDER BY id" + ) + instance2.query( + "CREATE TABLE IF NOT EXISTS local_table (id UInt32) ENGINE = MergeTree() ORDER BY id" + ) + instance2.query("INSERT INTO local_table VALUES (1), (2), (3)") + instance1.query( + "CREATE TABLE IF NOT EXISTS distributed_table AS local_table ENGINE = Distributed(test_ldap_cluster, default, local_table)" + ) + + result = instance1.query( + "SELECT sum(id) FROM distributed_table", user="johndoe", password="qwertz" + ) + assert result.strip() == "6" + + instance1.query("DROP TABLE IF EXISTS distributed_table SYNC") + instance1.query("DROP TABLE IF EXISTS local_table SYNC") + instance2.query("DROP TABLE IF EXISTS local_table SYNC") + instance2.query("DROP ROLE IF EXISTS role_read") + + delete_ldap_group(ldap_cluster, group_cn="clickhouse-role_read") + + +def test_remote_query_user_does_not_exist_locally(ldap_cluster): + """ + Check that even if user does not exist locally, using it to execute remote queries is still possible + """ + instance2.query("DROP USER IF EXISTS non_local") + instance2.query("DROP TABLE IF EXISTS test_table sync") + + instance2.query("CREATE USER non_local") + instance2.query("CREATE TABLE test_table (id Int16) ENGINE=Memory") + instance2.query("INSERT INTO test_table VALUES (123)") + instance2.query("GRANT SELECT ON default.test_table TO non_local") + + result = instance1.query( + "SELECT * FROM remote('instance2', 'default.test_table', 'non_local')" + ) + assert result.strip() == "123" + + instance2.query("DROP USER IF EXISTS non_local") + instance2.query("DROP TABLE IF EXISTS test_table SYNC") diff --git a/tests/integration/test_parallel_replicas_all_marks_read/test.py b/tests/integration/test_parallel_replicas_all_marks_read/test.py index 7776ccb0c095..e67da7fe02de 100644 --- a/tests/integration/test_parallel_replicas_all_marks_read/test.py +++ b/tests/integration/test_parallel_replicas_all_marks_read/test.py @@ -54,7 +54,7 @@ def _get_result_without_parallel_replicas(query): return nodes[0].query( query, settings={ - "allow_experimental_parallel_reading_from_replicas": 0, + "enable_parallel_replicas": 0, }, ) @@ -65,7 +65,7 @@ def _get_result_with_parallel_replicas( return nodes[0].query( query, settings={ - "allow_experimental_parallel_reading_from_replicas": 2, + "enable_parallel_replicas": 2, "max_parallel_replicas": 6, "cluster_for_parallel_replicas": f"{cluster_name}", "parallel_replicas_mark_segment_size": parallel_replicas_mark_segment_size, diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py index 375fe58d7417..39fd7ff292e6 100644 --- a/tests/integration/test_parallel_replicas_custom_key/test.py +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -37,13 +37,13 @@ def insert_data(table_name, row_num, all_nodes=False): @pytest.mark.parametrize("custom_key", ["sipHash64(key)", "key"]) -@pytest.mark.parametrize("filter_type", ["default", "range"]) +@pytest.mark.parametrize("parallel_replicas_mode", ["custom_key_sampling"]) @pytest.mark.parametrize( "cluster", ["test_multiple_shards_multiple_replicas", "test_single_shard_multiple_replicas"], ) def test_parallel_replicas_custom_key_distributed( - start_cluster, cluster, custom_key, filter_type + start_cluster, cluster, custom_key, parallel_replicas_mode ): for node in nodes: node.rotate_logs() @@ -84,7 +84,8 @@ def test_parallel_replicas_custom_key_distributed( settings={ "max_parallel_replicas": 4, "parallel_replicas_custom_key": custom_key, - "parallel_replicas_custom_key_filter_type": filter_type, + "enable_parallel_replicas": 1, + "parallel_replicas_mode": parallel_replicas_mode, "prefer_localhost_replica": 0, }, ) @@ -100,13 +101,15 @@ def test_parallel_replicas_custom_key_distributed( @pytest.mark.parametrize("custom_key", ["sipHash64(key)", "key"]) -@pytest.mark.parametrize("filter_type", ["default", "range"]) +@pytest.mark.parametrize( + "parallel_replicas_mode", ["custom_key_sampling", "custom_key_range"] +) @pytest.mark.parametrize( "cluster", ["test_single_shard_multiple_replicas"], ) def test_parallel_replicas_custom_key_mergetree( - start_cluster, cluster, custom_key, filter_type + start_cluster, cluster, custom_key, parallel_replicas_mode ): for node in nodes: node.rotate_logs() @@ -130,8 +133,9 @@ def test_parallel_replicas_custom_key_mergetree( "SELECT key, count() FROM test_table_for_mt GROUP BY key ORDER BY key", settings={ "max_parallel_replicas": 4, + "enable_parallel_replicas": 1, "parallel_replicas_custom_key": custom_key, - "parallel_replicas_custom_key_filter_type": filter_type, + "parallel_replicas_mode": parallel_replicas_mode, "parallel_replicas_for_non_replicated_merge_tree": 1, "cluster_for_parallel_replicas": cluster, }, @@ -141,13 +145,15 @@ def test_parallel_replicas_custom_key_mergetree( @pytest.mark.parametrize("custom_key", ["sipHash64(key)", "key"]) -@pytest.mark.parametrize("filter_type", ["default", "range"]) +@pytest.mark.parametrize( + "parallel_replicas_mode", ["custom_key_sampling", "custom_key_range"] +) @pytest.mark.parametrize( "cluster", ["test_single_shard_multiple_replicas"], ) def test_parallel_replicas_custom_key_replicatedmergetree( - start_cluster, cluster, custom_key, filter_type + start_cluster, cluster, custom_key, parallel_replicas_mode ): for node in nodes: node.rotate_logs() @@ -174,8 +180,9 @@ def test_parallel_replicas_custom_key_replicatedmergetree( "SELECT key, count() FROM test_table_for_rmt GROUP BY key ORDER BY key", settings={ "max_parallel_replicas": 4, + "enable_parallel_replicas": 1, "parallel_replicas_custom_key": custom_key, - "parallel_replicas_custom_key_filter_type": filter_type, + "parallel_replicas_mode": parallel_replicas_mode, "cluster_for_parallel_replicas": cluster, }, ) diff --git a/tests/integration/test_parallel_replicas_custom_key_failover/test.py b/tests/integration/test_parallel_replicas_custom_key_failover/test.py index f24a24f32387..d46f7955588c 100644 --- a/tests/integration/test_parallel_replicas_custom_key_failover/test.py +++ b/tests/integration/test_parallel_replicas_custom_key_failover/test.py @@ -52,13 +52,15 @@ def create_tables(cluster, table_name): @pytest.mark.parametrize("use_hedged_requests", [1, 0]) @pytest.mark.parametrize("custom_key", ["sipHash64(key)", "key"]) -@pytest.mark.parametrize("filter_type", ["default", "range"]) +@pytest.mark.parametrize( + "parallel_replicas_mode", ["custom_key_sampling", "custom_key_range"] +) @pytest.mark.parametrize("prefer_localhost_replica", [0, 1]) def test_parallel_replicas_custom_key_failover( start_cluster, use_hedged_requests, custom_key, - filter_type, + parallel_replicas_mode, prefer_localhost_replica, ): cluster_name = "test_single_shard_multiple_replicas" @@ -77,8 +79,9 @@ def test_parallel_replicas_custom_key_failover( settings={ "log_comment": log_comment, "max_parallel_replicas": 4, + "enable_parallel_replicas": 1, "parallel_replicas_custom_key": custom_key, - "parallel_replicas_custom_key_filter_type": filter_type, + "parallel_replicas_mode": parallel_replicas_mode, "use_hedged_requests": use_hedged_requests, "prefer_localhost_replica": prefer_localhost_replica, # avoid considering replica delay on connection choice diff --git a/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py b/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py index ddfd07a08645..3f462312967b 100644 --- a/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py +++ b/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py @@ -56,12 +56,14 @@ def create_tables(table_name): @pytest.mark.parametrize("use_hedged_requests", [1, 0]) @pytest.mark.parametrize("custom_key", ["sipHash64(key)", "key"]) -@pytest.mark.parametrize("filter_type", ["default", "range"]) +@pytest.mark.parametrize( + "parallel_replicas_mode", ["custom_key_sampling", "custom_key_range"] +) def test_parallel_replicas_custom_key_load_balancing( start_cluster, use_hedged_requests, custom_key, - filter_type, + parallel_replicas_mode, ): cluster_name = "test_single_shard_multiple_replicas" table = "test_table" @@ -80,8 +82,9 @@ def test_parallel_replicas_custom_key_load_balancing( "log_comment": log_comment, "prefer_localhost_replica": 0, "max_parallel_replicas": 4, + "enable_parallel_replicas": 1, "parallel_replicas_custom_key": custom_key, - "parallel_replicas_custom_key_filter_type": filter_type, + "parallel_replicas_mode": parallel_replicas_mode, "use_hedged_requests": use_hedged_requests, # avoid considering replica delay on connection choice # otherwise connection can be not distributed evenly among available nodes diff --git a/tests/integration/test_parallel_replicas_distributed_skip_shards/test.py b/tests/integration/test_parallel_replicas_distributed_skip_shards/test.py index 0f54d053ce1d..0c7e820114ac 100644 --- a/tests/integration/test_parallel_replicas_distributed_skip_shards/test.py +++ b/tests/integration/test_parallel_replicas_distributed_skip_shards/test.py @@ -85,7 +85,7 @@ def test_skip_unavailable_shards(start_cluster, prefer_localhost_replica): node1.query( f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", settings={ - "allow_experimental_parallel_reading_from_replicas": 2, + "enable_parallel_replicas": 2, "max_parallel_replicas": 3, "prefer_localhost_replica": prefer_localhost_replica, "skip_unavailable_shards": 1, @@ -119,7 +119,7 @@ def test_error_on_unavailable_shards(start_cluster, prefer_localhost_replica): node1.query( f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", settings={ - "allow_experimental_parallel_reading_from_replicas": 2, + "enable_parallel_replicas": 2, "max_parallel_replicas": 3, "prefer_localhost_replica": prefer_localhost_replica, "skip_unavailable_shards": 0, @@ -154,7 +154,7 @@ def test_no_unavailable_shards(start_cluster, skip_unavailable_shards): node1.query( f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", settings={ - "allow_experimental_parallel_reading_from_replicas": 2, + "enable_parallel_replicas": 2, "max_parallel_replicas": 3, "prefer_localhost_replica": 0, "skip_unavailable_shards": skip_unavailable_shards, diff --git a/tests/integration/test_parallel_replicas_failover/test.py b/tests/integration/test_parallel_replicas_failover/test.py index bf25136bff7f..340aa273811c 100644 --- a/tests/integration/test_parallel_replicas_failover/test.py +++ b/tests/integration/test_parallel_replicas_failover/test.py @@ -1,4 +1,5 @@ import pytest +import uuid from helpers.cluster import ClickHouseCluster @@ -72,7 +73,7 @@ def test_skip_replicas_without_table(start_cluster): node1.query( f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", settings={ - "allow_experimental_parallel_reading_from_replicas": 2, + "enable_parallel_replicas": 2, "max_parallel_replicas": 3, "cluster_for_parallel_replicas": cluster_name, "log_comment": log_comment, @@ -84,7 +85,7 @@ def test_skip_replicas_without_table(start_cluster): node1.query("SYSTEM FLUSH LOGS") assert ( node1.query( - f"SELECT ProfileEvents['DistributedConnectionMissingTable'], ProfileEvents['ParallelReplicasUnavailableCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '{log_comment}' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0" + f"SELECT ProfileEvents['DistributedConnectionMissingTable'], ProfileEvents['ParallelReplicasUnavailableCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '{log_comment}' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS enable_parallel_replicas=0" ) == "1\t1\n" ) @@ -105,7 +106,7 @@ def test_skip_unresponsive_replicas(start_cluster): node1.query( f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", settings={ - "allow_experimental_parallel_reading_from_replicas": 2, + "enable_parallel_replicas": 2, "max_parallel_replicas": 3, "cluster_for_parallel_replicas": cluster_name, }, diff --git a/tests/integration/test_parallel_replicas_invisible_parts/test.py b/tests/integration/test_parallel_replicas_invisible_parts/test.py index cab3fb46fe92..bc0c96fc56c1 100644 --- a/tests/integration/test_parallel_replicas_invisible_parts/test.py +++ b/tests/integration/test_parallel_replicas_invisible_parts/test.py @@ -81,7 +81,7 @@ def _get_result_with_parallel_replicas( return nodes[0].query( query, settings={ - "allow_experimental_parallel_reading_from_replicas": 2, + "enable_parallel_replicas": 2, "max_parallel_replicas": len(nodes), "cluster_for_parallel_replicas": f"{cluster_name}", "parallel_replicas_mark_segment_size": parallel_replicas_mark_segment_size, diff --git a/tests/integration/test_parallel_replicas_no_replicas/test.py b/tests/integration/test_parallel_replicas_no_replicas/test.py index 62d4b005d94c..4c25315202c5 100644 --- a/tests/integration/test_parallel_replicas_no_replicas/test.py +++ b/tests/integration/test_parallel_replicas_no_replicas/test.py @@ -45,9 +45,10 @@ def test_skip_all_replicas( initiator.query( f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", settings={ - "allow_experimental_parallel_reading_from_replicas": 2, + "enable_parallel_replicas": 2, "max_parallel_replicas": max_parallel_replicas, "cluster_for_parallel_replicas": cluster_name, "skip_unavailable_shards": skip_unavailable_shards, + "parallel_replicas_local_plan": 0, }, ) diff --git a/tests/integration/test_parallel_replicas_over_distributed/test.py b/tests/integration/test_parallel_replicas_over_distributed/test.py index 3d35ec3fa8e1..2752f1cb00f5 100644 --- a/tests/integration/test_parallel_replicas_over_distributed/test.py +++ b/tests/integration/test_parallel_replicas_over_distributed/test.py @@ -137,7 +137,7 @@ def test_parallel_replicas_over_distributed( node.query( f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", settings={ - "allow_experimental_parallel_reading_from_replicas": 2, + "enable_parallel_replicas": 2, "prefer_localhost_replica": prefer_localhost_replica, "max_parallel_replicas": max_parallel_replicas, }, @@ -150,7 +150,7 @@ def test_parallel_replicas_over_distributed( node.query( f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", settings={ - "allow_experimental_parallel_reading_from_replicas": 0, + "enable_parallel_replicas": 0, }, ) == expected_result diff --git a/tests/integration/test_parallel_replicas_protocol/__init__.py b/tests/integration/test_parallel_replicas_protocol/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_parallel_replicas_protocol/configs/remote_servers.xml b/tests/integration/test_parallel_replicas_protocol/configs/remote_servers.xml new file mode 100644 index 000000000000..9fc2f119898a --- /dev/null +++ b/tests/integration/test_parallel_replicas_protocol/configs/remote_servers.xml @@ -0,0 +1,21 @@ + + + + + false + + node0 + 9000 + + + node1 + 9000 + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_parallel_replicas_protocol/test.py b/tests/integration/test_parallel_replicas_protocol/test.py new file mode 100644 index 000000000000..c43226c5ecc9 --- /dev/null +++ b/tests/integration/test_parallel_replicas_protocol/test.py @@ -0,0 +1,74 @@ +import pytest +import re + +from helpers.cluster import ClickHouseCluster +from random import randint + +cluster = ClickHouseCluster(__file__) + +nodes = [ + cluster.add_instance( + f"node{num}", + main_configs=["configs/remote_servers.xml"], + with_zookeeper=True, + macros={"replica": f"node{num}", "shard": "shard"}, + ) + 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): + nodes[0].query( + f"DROP TABLE IF EXISTS {table_name} ON CLUSTER 'parallel_replicas'", + settings={"database_atomic_wait_for_drop_and_detach_synchronously": True}, + ) + + # big number of granules + low total size in bytes = super tiny granules = big min_marks_per_task + # => big mark_segment_size will be chosen. it is not required to be big, just not equal to the default + nodes[0].query( + f""" + CREATE TABLE {table_name} ON CLUSTER 'parallel_replicas' (value Int64) + Engine=ReplicatedMergeTree('/test_parallel_replicas/shard/{table_name}', '{{replica}}') + ORDER BY () + SETTINGS index_granularity = 1 + """ + ) + + nodes[0].query(f"INSERT INTO {table_name} SELECT 42 FROM numbers(1000)") + nodes[0].query(f"SYSTEM SYNC REPLICA ON CLUSTER 'parallel_replicas' {table_name}") + + +# now mark_segment_size is part of the protocol and is communicated to the initiator. +# let's check that the correct value is actually used by the coordinator +def test_mark_segment_size_communicated_correctly(start_cluster): + table_name = "t" + _create_tables(table_name) + + for local_plan in [0, 1]: + query_id = f"query_id_{randint(0, 1000000)}" + nodes[0].query( + f"SELECT sum(value) FROM {table_name}", + settings={ + "allow_experimental_parallel_reading_from_replicas": 2, + "max_parallel_replicas": 100, + "cluster_for_parallel_replicas": "parallel_replicas", + "parallel_replicas_mark_segment_size": 0, + "parallel_replicas_local_plan": local_plan, + "query_id": query_id, + }, + ) + + nodes[0].query("SYSTEM FLUSH LOGS") + log_line = nodes[0].grep_in_log( + f"{query_id}.*Reading state is fully initialized" + ) + assert re.search(r"mark_segment_size: (\d+)", log_line).group(1) == "16384" diff --git a/tests/integration/test_parallel_replicas_skip_shards/test.py b/tests/integration/test_parallel_replicas_skip_shards/test.py index a18c82a53a94..0a0a16915428 100644 --- a/tests/integration/test_parallel_replicas_skip_shards/test.py +++ b/tests/integration/test_parallel_replicas_skip_shards/test.py @@ -25,7 +25,7 @@ def test_skip_unavailable_shards(start_cluster): node1.query( "SELECT hostName() as h FROM clusterAllReplicas('two_shards', system.one) order by h", settings={ - "allow_experimental_parallel_reading_from_replicas": 0, + "enable_parallel_replicas": 0, "skip_unavailable_shards": 1, }, ) @@ -36,7 +36,7 @@ def test_skip_unavailable_shards(start_cluster): node1.query( "SELECT hostName() as h FROM clusterAllReplicas('two_shards', system.one) order by h", settings={ - "allow_experimental_parallel_reading_from_replicas": 2, + "enable_parallel_replicas": 2, "max_parallel_replicas": 3, "skip_unavailable_shards": 1, # "async_socket_for_remote" : 0, @@ -53,7 +53,7 @@ def test_error_on_unavailable_shards(start_cluster): node1.query( "SELECT hostName() as h FROM clusterAllReplicas('two_shards', system.one) order by h", settings={ - "allow_experimental_parallel_reading_from_replicas": 0, + "enable_parallel_replicas": 0, "skip_unavailable_shards": 0, }, ) @@ -62,7 +62,7 @@ def test_error_on_unavailable_shards(start_cluster): node1.query( "SELECT hostName() as h FROM clusterAllReplicas('two_shards', system.one) order by h", settings={ - "allow_experimental_parallel_reading_from_replicas": 2, + "enable_parallel_replicas": 2, "max_parallel_replicas": 3, "skip_unavailable_shards": 0, }, diff --git a/tests/integration/test_parallel_replicas_snapshot_from_initiator/__init__.py b/tests/integration/test_parallel_replicas_snapshot_from_initiator/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_parallel_replicas_snapshot_from_initiator/configs/remote_servers.xml b/tests/integration/test_parallel_replicas_snapshot_from_initiator/configs/remote_servers.xml new file mode 100644 index 000000000000..734acf5f3639 --- /dev/null +++ b/tests/integration/test_parallel_replicas_snapshot_from_initiator/configs/remote_servers.xml @@ -0,0 +1,33 @@ + + + + + false + + node0 + 9000 + + + node1 + 9000 + + + node2 + 9000 + + + node3 + 9000 + + + node4 + 9000 + + + node5 + 9000 + + + + + diff --git a/tests/integration/test_parallel_replicas_snapshot_from_initiator/test.py b/tests/integration/test_parallel_replicas_snapshot_from_initiator/test.py new file mode 100644 index 000000000000..a7e7e99455bc --- /dev/null +++ b/tests/integration/test_parallel_replicas_snapshot_from_initiator/test.py @@ -0,0 +1,73 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +nodes = [ + cluster.add_instance( + f"node{num}", main_configs=["configs/remote_servers.xml"], with_zookeeper=True + ) + for num in range(6) +] + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def _create_tables(table_name): + for idx, node in enumerate(nodes): + node.query( + f"DROP TABLE IF EXISTS {table_name}", + settings={"database_atomic_wait_for_drop_and_detach_synchronously": True}, + ) + + node.query( + f""" + CREATE TABLE {table_name} (value Int64) + Engine=ReplicatedMergeTree('/test_parallel_replicas/shard/{table_name}', '{idx}') + ORDER BY () + """ + ) + + nodes[0].query( + f"INSERT INTO {table_name} SELECT * FROM numbers(1000)", + settings={"insert_deduplicate": 0}, + ) + nodes[0].query(f"SYSTEM SYNC REPLICA ON CLUSTER 'parallel_replicas' {table_name}") + + for idx, node in enumerate(nodes): + node.query("SYSTEM STOP REPLICATED SENDS") + # the same data on all nodes except for a single value + node.query( + f"INSERT INTO {table_name} VALUES ({idx})", + settings={"insert_deduplicate": 0}, + ) + + +# check that we use the state of data parts from the initiator node (for some sort of determinism of what is been read). +# currently it is implemented only when we build local plan for the initiator node (we aim to make this behavior default) +def test_initiator_snapshot_is_used_for_reading(start_cluster): + table_name = "t" + _create_tables(table_name) + + for idx, node in enumerate(nodes): + expected = 499500 + idx # sum of all integers 0..999 + idx + assert ( + node.query( + f"SELECT sum(value) FROM {table_name}", + settings={ + "allow_experimental_parallel_reading_from_replicas": 2, + "max_parallel_replicas": 100, + "cluster_for_parallel_replicas": "parallel_replicas", + "parallel_replicas_local_plan": True, + }, + ) + == f"{expected}\n" + ) diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml index e69df7d39f65..469e76528c5f 100644 --- a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml @@ -2,7 +2,7 @@ 1 - 1 + 1 1 default 100 diff --git a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql index 901b818cbc06..fa713fc8d43a 100644 --- a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql +++ b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql @@ -1,7 +1,9 @@ -- Tags: replica, distributed -SET allow_experimental_parallel_reading_from_replicas = 0; +SET enable_parallel_replicas = 1; +SET parallel_replicas_mode = 'sampling_key'; SET max_parallel_replicas = 2; +SET parallel_replicas_for_non_replicated_merge_tree = 1; DROP TABLE IF EXISTS report; set allow_deprecated_syntax_for_merge_tree=1; diff --git a/tests/queries/0_stateless/00193_parallel_replicas.sql b/tests/queries/0_stateless/00193_parallel_replicas.sql index 2549ada2a78a..b19d9cfdefe4 100644 --- a/tests/queries/0_stateless/00193_parallel_replicas.sql +++ b/tests/queries/0_stateless/00193_parallel_replicas.sql @@ -13,12 +13,14 @@ INSERT INTO parallel_replicas (x, u, s) VALUES (41, 42, 'U'),(43, 44, 'V'),(45, INSERT INTO parallel_replicas (x, u, s) VALUES (51, 52, 'Z'); /* - * Проверяем, что: - * - на каждой реплике таблица не пустая; - * - объединение данных всех реплик совпадает с содержанием таблицы parallel_replicas. - */ +* Check that: +* - the table is not empty on each replica; +* - combining the data of all replicas coincides with the contents of the parallel_replicas table. +*/ -/* Две реплики */ +/* Two replicas */ + +SET enable_parallel_replicas=1, parallel_replicas_mode='sampling_key', max_parallel_replicas=3, parallel_replicas_for_non_replicated_merge_tree = 1; CREATE TABLE parallel_replicas_backup(d Date DEFAULT today(), x UInt32, u UInt64, s String) ENGINE = Memory; @@ -38,7 +40,7 @@ SELECT x, u, s FROM parallel_replicas_backup ORDER BY x, u, s ASC; DROP TABLE parallel_replicas_backup; CREATE TABLE parallel_replicas_backup(d Date DEFAULT today(), x UInt32, u UInt64, s String) ENGINE = Memory; -/* Три реплики */ +/* Three replicas */ SET parallel_replicas_count = 3; diff --git a/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql b/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql index 86c84427297d..566660f67c63 100644 --- a/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql +++ b/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql @@ -1,7 +1,5 @@ -- Tags: replica, distributed -set allow_experimental_parallel_reading_from_replicas=0; - drop table if exists test_max_parallel_replicas_lr; -- If you wonder why the table is named with "_lr" suffix in this test. @@ -10,7 +8,10 @@ drop table if exists test_max_parallel_replicas_lr; CREATE TABLE test_max_parallel_replicas_lr (timestamp UInt64) ENGINE = MergeTree ORDER BY (intHash32(timestamp)) SAMPLE BY intHash32(timestamp); INSERT INTO test_max_parallel_replicas_lr select number as timestamp from system.numbers limit 100; +SET enable_parallel_replicas = 1; +SET parallel_replicas_mode='sampling_key'; SET max_parallel_replicas = 2; +SET parallel_replicas_for_non_replicated_merge_tree = 1; select count() FROM remote('127.0.0.{2|3}', currentDatabase(), test_max_parallel_replicas_lr) PREWHERE timestamp > 0; drop table test_max_parallel_replicas_lr; diff --git a/tests/queries/0_stateless/01034_sample_final_distributed.sql b/tests/queries/0_stateless/01034_sample_final_distributed.sql index 64bafd17b2d9..bbb1b0dcf691 100644 --- a/tests/queries/0_stateless/01034_sample_final_distributed.sql +++ b/tests/queries/0_stateless/01034_sample_final_distributed.sql @@ -1,6 +1,9 @@ -- Tags: distributed -set allow_experimental_parallel_reading_from_replicas = 0; +set enable_parallel_replicas = 1; +set parallel_replicas_mode = 'sampling_key'; +set max_parallel_replicas = 3; +set parallel_replicas_for_non_replicated_merge_tree = 1; drop table if exists sample_final; create table sample_final (CounterID UInt32, EventDate Date, EventTime DateTime, UserID UInt64, Sign Int8) engine = CollapsingMergeTree(Sign) order by (CounterID, EventDate, intHash32(UserID), EventTime) sample by intHash32(UserID) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; diff --git a/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql b/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql index ec650551bafb..f31929306998 100644 --- a/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql +++ b/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql @@ -2,7 +2,7 @@ -- set distributed_foreground_insert = 1; -- see https://github.com/ClickHouse/ClickHouse/issues/18971 -SET allow_experimental_parallel_reading_from_replicas = 0; -- see https://github.com/ClickHouse/ClickHouse/issues/34525 +SET enable_parallel_replicas = 0; -- see https://github.com/ClickHouse/ClickHouse/issues/34525 SET prefer_localhost_replica = 1; DROP TABLE IF EXISTS local_01099_a; diff --git a/tests/queries/0_stateless/01517_select_final_distributed.sql b/tests/queries/0_stateless/01517_select_final_distributed.sql index 701828b0b38f..d2c6e63a861a 100644 --- a/tests/queries/0_stateless/01517_select_final_distributed.sql +++ b/tests/queries/0_stateless/01517_select_final_distributed.sql @@ -1,16 +1,16 @@ -- Tags: distributed -SET allow_experimental_parallel_reading_from_replicas = 0; +SET enable_parallel_replicas = 0; DROP TABLE IF EXISTS test5346; -CREATE TABLE test5346 (`Id` String, `Timestamp` DateTime, `updated` DateTime) +CREATE TABLE test5346 (`Id` String, `Timestamp` DateTime, `updated` DateTime) ENGINE = ReplacingMergeTree(updated) PARTITION BY tuple() ORDER BY (Timestamp, Id); INSERT INTO test5346 VALUES('1',toDateTime('2020-01-01 00:00:00'),toDateTime('2020-01-01 00:00:00')); -SELECT Id, Timestamp -FROM remote('localhost,127.0.0.1,127.0.0.2',currentDatabase(),'test5346') FINAL +SELECT Id, Timestamp +FROM remote('localhost,127.0.0.1,127.0.0.2',currentDatabase(),'test5346') FINAL ORDER BY Timestamp; SELECT Id, Timestamp diff --git a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql index 04777f5b31ce..3704119c989b 100644 --- a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql +++ b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql @@ -1,6 +1,9 @@ -- Tags: replica -SET allow_experimental_parallel_reading_from_replicas=0; +SET enable_parallel_replicas=1; +SET max_parallel_replicas=3; +SET parallel_replicas_mode='sampling_key'; +SET parallel_replicas_for_non_replicated_merge_tree = 1; DROP TABLE IF EXISTS t; CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY x; diff --git a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh index 3c44a2a7ba7f..5c801af531de 100755 --- a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh +++ b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 -nm < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null +${CLICKHOUSE_CLIENT} --enable_parallel_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 -m < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null diff --git a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh index bc90f4b2c111..347293036e17 100755 --- a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh +++ b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh @@ -14,7 +14,7 @@ as select * from numbers(1); # # Logical error: 'Coordinator for parallel reading from replicas is not initialized'. opts=( - --allow_experimental_parallel_reading_from_replicas 1 + --enable_parallel_replicas 1 --parallel_replicas_for_non_replicated_merge_tree 1 --max_parallel_replicas 3 --cluster_for_parallel_replicas parallel_replicas diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.reference b/tests/queries/0_stateless/02404_memory_bound_merging.reference index 0409c48f846a..e02684c3b95d 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.reference +++ b/tests/queries/0_stateless/02404_memory_bound_merging.reference @@ -100,7 +100,7 @@ select a, count() from dist_t_different_dbs group by a, b order by a limit 5 off 504 2000 1000000 -- { echoOn } -- -explain pipeline select a from pr_t group by a order by a limit 5 offset 500; +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings parallel_replicas_local_plan=0; (Expression) ExpressionTransform (Limit) @@ -119,6 +119,31 @@ ExpressionTransform Resize 1 → 4 GroupingAggregatedTransform 3 → 1 (ReadFromRemoteParallelReplicas) +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=1, parallel_replicas_local_plan=1; +(Expression) +ExpressionTransform + (Limit) + Limit + (Sorting) + MergingSortedTransform 4 → 1 + MergeSortingTransform × 4 + LimitsCheckingTransform × 4 + PartialSortingTransform × 4 + (Expression) + ExpressionTransform × 4 + (MergingAggregated) + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + FinishAggregatingInOrderTransform 3 → 1 + (Union) + (Aggregating) + FinalizeAggregatedTransform + AggregatingInOrderTransform + (Expression) + ExpressionTransform + (ReadFromMergeTree) + MergeTreeSelect(pool: ReadPoolParallelReplicasInOrder, algorithm: InOrder) 0 → 1 + (ReadFromRemoteParallelReplicas) select a, count() from pr_t group by a order by a limit 5 offset 500; 500 1000 501 1000 diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.sql b/tests/queries/0_stateless/02404_memory_bound_merging.sql index a2de19dff8a7..866f49717aba 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.sql +++ b/tests/queries/0_stateless/02404_memory_bound_merging.sql @@ -63,17 +63,17 @@ drop table if exists pr_t; create table pr_t(a UInt64, b UInt64) engine=MergeTree order by a; insert into pr_t select number % 1000, number % 1000 from numbers_mt(1e6); -set allow_experimental_parallel_reading_from_replicas = 1; +set enable_parallel_replicas = 1; set parallel_replicas_for_non_replicated_merge_tree = 1; set max_parallel_replicas = 3; -set use_hedged_requests = 0; set cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; set distributed_aggregation_memory_efficient=1; select count() from pr_t; -- { echoOn } -- -explain pipeline select a from pr_t group by a order by a limit 5 offset 500; +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings parallel_replicas_local_plan=0; +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=1, parallel_replicas_local_plan=1; select a, count() from pr_t group by a order by a limit 5 offset 500; select a, count() from pr_t group by a, b order by a limit 5 offset 500; diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.reference b/tests/queries/0_stateless/02532_send_logs_level_test.reference index 72f4ea06184f..e69de29bb2d1 100644 --- a/tests/queries/0_stateless/02532_send_logs_level_test.reference +++ b/tests/queries/0_stateless/02532_send_logs_level_test.reference @@ -1,4 +0,0 @@ - MergeTreeReadPoolBase: Will use min_marks_per_task=24 - MergeTreeMarksLoader: Loading marks from path data.cmrk3 - MergeTreeRangeReader: First reader returned: num_rows: 1, columns: 1, total_rows_per_granule: 1, no filter, column[0]: Int32(size = 1), requested columns: key - MergeTreeRangeReader: read() returned num_rows: 1, columns: 1, total_rows_per_granule: 1, no filter, column[0]: Int32(size = 1), sample block key diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.sh b/tests/queries/0_stateless/02532_send_logs_level_test.sh index a50539311cba..a91e49ddd229 100755 --- a/tests/queries/0_stateless/02532_send_logs_level_test.sh +++ b/tests/queries/0_stateless/02532_send_logs_level_test.sh @@ -18,6 +18,10 @@ $CLICKHOUSE_CLIENT -m -q " # instead of "last" value, hence you cannot simply append another # --send_logs_level here. CLICKHOUSE_CLIENT_CLEAN=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=test/g') -$CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& grep -o -e '.*' -e '.*' -$CLICKHOUSE_CLIENT -q "drop table data" +set -e + +trap '$CLICKHOUSE_CLIENT -q "drop table data"' EXIT + +$CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& (! grep -q -o -e '.*') +$CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& grep -q -o -e '.*' diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.reference b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.reference index 1bb07f0d9168..4759240cfae0 100644 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.reference +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.reference @@ -1,176 +1,176 @@ query='SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_mt)' with custom_key='sipHash64(x)' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 Hello -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 Hello -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 Hello -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 Hello -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 Hello -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 Hello query='SELECT * FROM 02535_custom_key_mt' with custom_key='sipHash64(x)' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 Hello -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 Hello -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 Hello -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 Hello -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 Hello -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 Hello query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_mt) GROUP BY y ORDER BY y' with custom_key='y' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 0 334 1 333 2 333 -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 0 334 1 333 2 333 -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 0 334 1 333 2 333 -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 0 334 1 333 2 333 -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 0 334 1 333 2 333 -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 0 334 1 333 2 333 query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_mt) GROUP BY y ORDER BY y' with custom_key='cityHash64(y)' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 0 334 1 333 2 333 -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 0 334 1 333 2 333 -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 0 334 1 333 2 333 -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 0 334 1 333 2 333 -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 0 334 1 333 2 333 -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 0 334 1 333 2 333 query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_mt) GROUP BY y ORDER BY y' with custom_key='cityHash64(y) + 1' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 0 334 1 333 2 333 -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 0 334 1 333 2 333 -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 0 334 1 333 2 333 -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 0 334 1 333 2 333 -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 0 334 1 333 2 333 -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 0 334 1 333 2 333 query='SELECT y, count() FROM 02535_custom_key_mt GROUP BY y ORDER BY y' with custom_key='y' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 0 334 1 333 2 333 -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 0 334 1 333 2 333 -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 0 334 1 333 2 333 -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 0 334 1 333 2 333 -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 0 334 1 333 2 333 -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 0 334 1 333 2 333 query='SELECT y, count() FROM 02535_custom_key_mt GROUP BY y ORDER BY y' with custom_key='cityHash64(y)' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 0 334 1 333 2 333 -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 0 334 1 333 2 333 -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 0 334 1 333 2 333 -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 0 334 1 333 2 333 -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 0 334 1 333 2 333 -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 0 334 1 333 2 333 query='SELECT y, count() FROM 02535_custom_key_mt GROUP BY y ORDER BY y' with custom_key='cityHash64(y) + 1' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 0 334 1 333 2 333 -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 0 334 1 333 2 333 -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 0 334 1 333 2 333 -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 0 334 1 333 2 333 -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 0 334 1 333 2 333 -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 0 334 1 333 2 333 diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.sh index fad43ea90706..aa27af090638 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.sh @@ -7,12 +7,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function run_with_custom_key { echo "query='$1' with custom_key='$2'" - for filter_type in 'default' 'range'; do + for filter_type in 'custom_key_sampling' 'custom_key_range'; do for max_replicas in {1..3}; do echo "filter_type='$filter_type' max_replicas=$max_replicas" query="$1 SETTINGS max_parallel_replicas=$max_replicas\ +, enable_parallel_replicas='1' \ +, parallel_replicas_mode ='$filter_type'\ , parallel_replicas_custom_key='$2'\ -, parallel_replicas_custom_key_filter_type='$filter_type'\ , parallel_replicas_for_non_replicated_merge_tree=1 \ , cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'" $CLICKHOUSE_CLIENT --query="$query" diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.reference b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.reference index c6526b506d34..170a51f1f255 100644 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.reference +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.reference @@ -1,176 +1,176 @@ query='SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_rmt)' with custom_key='sipHash64(x)' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 Hello -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 Hello -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 Hello -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 Hello -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 Hello -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 Hello query='SELECT * FROM 02535_custom_key_rmt' with custom_key='sipHash64(x)' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 Hello -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 Hello -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 Hello -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 Hello -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 Hello -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 Hello query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_rmt_hash) GROUP BY y ORDER BY y' with custom_key='y' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 0 334 1 333 2 333 -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 0 334 1 333 2 333 -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 0 334 1 333 2 333 -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 0 334 1 333 2 333 -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 0 334 1 333 2 333 -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 0 334 1 333 2 333 query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_rmt_hash) GROUP BY y ORDER BY y' with custom_key='cityHash64(y)' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 0 334 1 333 2 333 -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 0 334 1 333 2 333 -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 0 334 1 333 2 333 -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 0 334 1 333 2 333 -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 0 334 1 333 2 333 -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 0 334 1 333 2 333 query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_rmt_hash) GROUP BY y ORDER BY y' with custom_key='cityHash64(y) + 1' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 0 334 1 333 2 333 -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 0 334 1 333 2 333 -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 0 334 1 333 2 333 -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 0 334 1 333 2 333 -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 0 334 1 333 2 333 -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 0 334 1 333 2 333 query='SELECT y, count() FROM 02535_custom_key_rmt_hash GROUP BY y ORDER BY y' with custom_key='y' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 0 334 1 333 2 333 -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 0 334 1 333 2 333 -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 0 334 1 333 2 333 -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 0 334 1 333 2 333 -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 0 334 1 333 2 333 -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 0 334 1 333 2 333 query='SELECT y, count() FROM 02535_custom_key_rmt_hash GROUP BY y ORDER BY y' with custom_key='cityHash64(y)' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 0 334 1 333 2 333 -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 0 334 1 333 2 333 -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 0 334 1 333 2 333 -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 0 334 1 333 2 333 -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 0 334 1 333 2 333 -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 0 334 1 333 2 333 query='SELECT y, count() FROM 02535_custom_key_rmt_hash GROUP BY y ORDER BY y' with custom_key='cityHash64(y) + 1' -filter_type='default' max_replicas=1 +filter_type='custom_key_sampling' max_replicas=1 0 334 1 333 2 333 -filter_type='default' max_replicas=2 +filter_type='custom_key_sampling' max_replicas=2 0 334 1 333 2 333 -filter_type='default' max_replicas=3 +filter_type='custom_key_sampling' max_replicas=3 0 334 1 333 2 333 -filter_type='range' max_replicas=1 +filter_type='custom_key_range' max_replicas=1 0 334 1 333 2 333 -filter_type='range' max_replicas=2 +filter_type='custom_key_range' max_replicas=2 0 334 1 333 2 333 -filter_type='range' max_replicas=3 +filter_type='custom_key_range' max_replicas=3 0 334 1 333 2 333 diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.sh index 6350f5027f90..87899aab274d 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.sh @@ -7,12 +7,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function run_with_custom_key { echo "query='$1' with custom_key='$2'" - for filter_type in 'default' 'range'; do + for filter_type in 'custom_key_sampling' 'custom_key_range'; do for max_replicas in {1..3}; do echo "filter_type='$filter_type' max_replicas=$max_replicas" query="$1 SETTINGS max_parallel_replicas=$max_replicas\ +, enable_parallel_replicas='1' \ +, parallel_replicas_mode ='$filter_type'\ , parallel_replicas_custom_key='$2'\ -, parallel_replicas_custom_key_filter_type='$filter_type'\ , parallel_replicas_for_non_replicated_merge_tree=1 \ , cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'" $CLICKHOUSE_CLIENT --query="$query" diff --git a/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql b/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql index ff7b53ce01ff..4179b0491fdd 100644 --- a/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql +++ b/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql @@ -1,4 +1,4 @@ CREATE TABLE IF NOT EXISTS t_02708(x DateTime) ENGINE = MergeTree ORDER BY tuple(); SET send_logs_level='error'; -SELECT count() FROM t_02708 SETTINGS allow_experimental_parallel_reading_from_replicas=1; +SELECT count() FROM t_02708 SETTINGS enable_parallel_replicas=1; DROP TABLE t_02708; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference index 125fa524c4ae..c6a3bbc3dfb1 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference @@ -31,10 +31,10 @@ 29 2j&S)ba?XG QuQj 17163829389637435056 3 UlI+1 14144472852965836438 =============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE =============== -0 3 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\') +0 2 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\') 0 3 SELECT `key`, `value1`, `value2`, toUInt64(min(`time`)) AS `start_ts` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` ORDER BY `key` ASC, `value1` ASC, `value2` ASC LIMIT 10 -1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0; -1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=1; +1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS enable_parallel_replicas = 1, enable_analyzer=0; +1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS enable_parallel_replicas = 1, enable_analyzer=1; =============== OUTER QUERY (NO PARALLEL) =============== >T%O ,z< 10 NQTpY# W\\Xx4 10 @@ -58,8 +58,8 @@ U c 10 UlI+1 10 bX?}ix [ Ny]2 G 10 t _CAST(1610517366120, \'UInt64\')) GROUP BY `__table3`.`key`, `__table3`.`value1`, `__table3`.`value2`) AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2` +0 2 SELECT `__table2`.`value1` AS `value1`, `__table2`.`value2` AS `value2`, count() AS `count` FROM `default`.`join_outer_table` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`key` AS `key`, `__table3`.`value1` AS `value1`, `__table3`.`value2` AS `value2` FROM `default`.`join_inner_table` AS `__table3` PREWHERE (`__table3`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table3`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table3`.`key`, `__table3`.`value1`, `__table3`.`value2`) AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2` 0 3 SELECT `key`, `value1`, `value2` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` 0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_` USING (`key`) GROUP BY `key`, `value1`, `value2` -1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0; -1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=1; +1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS enable_parallel_replicas = 1, enable_analyzer=0; +1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS enable_parallel_replicas = 1, enable_analyzer=1; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql index 8121d60a05be..2cf1ffa13e8d 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql @@ -22,8 +22,8 @@ SELECT LIMIT 100; SET max_parallel_replicas = 3; -SET prefer_localhost_replica = 1; SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; +SET parallel_replicas_local_plan = 1; SET joined_subquery_requires_alias = 0; SELECT '=============== INNER QUERY (NO PARALLEL) ==============='; @@ -64,7 +64,7 @@ PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1 GROUP BY key, value1, value2 ORDER BY key, value1, value2 LIMIT 10 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0; +SETTINGS enable_parallel_replicas = 1, enable_analyzer=0; -- Parallel inner query alone SELECT @@ -77,7 +77,7 @@ PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1 GROUP BY key, value1, value2 ORDER BY key, value1, value2 LIMIT 10 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=1; +SETTINGS enable_parallel_replicas = 1, enable_analyzer=1; SELECT '=============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE ==============='; @@ -184,7 +184,7 @@ FROM ) GROUP BY value1, value2 ORDER BY value1, value2 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0; +SETTINGS enable_parallel_replicas = 1, enable_analyzer=0; -- Parallel full query SELECT @@ -214,7 +214,7 @@ FROM ) GROUP BY value1, value2 ORDER BY value1, value2 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=1; +SETTINGS enable_parallel_replicas = 1, enable_analyzer=1; SYSTEM FLUSH LOGS; diff --git a/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql b/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql index a7112e5484b1..ca22f324f2b5 100644 --- a/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql +++ b/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql @@ -18,7 +18,7 @@ INSERT INTO join_inner_table__fuzz_1 SELECT FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) LIMIT 100; -SET max_parallel_replicas = 3, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree=1; +SET max_parallel_replicas = 3, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_parallel_replicas = 1, parallel_replicas_for_non_replicated_merge_tree=1; -- SELECT query will write a Warning to the logs SET send_logs_level='error'; diff --git a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql index e166ce9b2849..b26cbc3cfad9 100644 --- a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql +++ b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS parallel_replicas_plain; CREATE TABLE parallel_replicas_plain (x String) ENGINE=MergeTree() ORDER BY x; INSERT INTO parallel_replicas_plain SELECT toString(number) FROM numbers(10); -SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, cluster_for_parallel_replicas='parallel_replicas'; +SET max_parallel_replicas=3, enable_parallel_replicas=1, cluster_for_parallel_replicas='parallel_replicas'; SET send_logs_level='error'; SET parallel_replicas_for_non_replicated_merge_tree = 0; diff --git a/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql b/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql index 6c121802b063..6acfb2482d38 100644 --- a/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql +++ b/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql @@ -2,12 +2,12 @@ CREATE TABLE IF NOT EXISTS parallel_replicas_final (x String) ENGINE=ReplacingMe INSERT INTO parallel_replicas_final SELECT toString(number) FROM numbers(10); -SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, cluster_for_parallel_replicas='parallel_replicas'; +SET max_parallel_replicas=3, enable_parallel_replicas=1, cluster_for_parallel_replicas='parallel_replicas'; SET parallel_replicas_for_non_replicated_merge_tree = 1; SELECT * FROM parallel_replicas_final FINAL FORMAT Null; -SET allow_experimental_parallel_reading_from_replicas=2; +SET enable_parallel_replicas=2; SELECT * FROM parallel_replicas_final FINAL FORMAT Null; -- { serverError SUPPORT_IS_DISABLED } diff --git a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql index 1a75e000349c..1b015ac495e6 100644 --- a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql +++ b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql @@ -2,13 +2,15 @@ DROP TABLE IF EXISTS test_parallel_replicas_unavailable_shards; CREATE TABLE test_parallel_replicas_unavailable_shards (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); INSERT INTO test_parallel_replicas_unavailable_shards SELECT * FROM numbers(10); -SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1; +SET enable_parallel_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1; SET send_logs_level='error'; -SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79'; +-- with local plan for initiator, the query can be executed fast on initator, we can simply not come to the point where unavailable replica can be detected +-- therefore disable local plan for now +SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79', parallel_replicas_local_plan=0; SYSTEM FLUSH LOGS; -SET allow_experimental_parallel_reading_from_replicas=0; +SET enable_parallel_replicas=0; SELECT ProfileEvents['ParallelReplicasUnavailableCount'] FROM system.query_log WHERE yesterday() <= event_date AND query_id in (select query_id from system.query_log where log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79' and current_database = currentDatabase()) and type = 'QueryFinish' and query_id == initial_query_id; DROP TABLE test_parallel_replicas_unavailable_shards; diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference index f60f1e0a3768..b94774e6ca3a 100644 --- a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference @@ -8,5 +8,5 @@ 5935810273536892891 7885388429666205427 8124171311239967992 -1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n enable_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1; -0 2 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1` +1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n enable_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n enable_parallel_replicas = 1; +0 1 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1` diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql index e60049f2756a..081077ba4601 100644 --- a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql @@ -1,5 +1,5 @@ -- Tags: zookeeper - +DROP TABLE IF EXISTS join_inner_table__fuzz_146_replicated; CREATE TABLE join_inner_table__fuzz_146_replicated ( `id` UUID, @@ -17,6 +17,8 @@ INSERT INTO join_inner_table__fuzz_146_replicated SELECT CAST('833c9e22-c245-4eb5-8745-117a9a1f26b1', 'UUID') AS id, CAST(rowNumberInAllBlocks(), 'String') AS key, * FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) LIMIT 10; +SET parallel_replicas_local_plan = 1; + -- Simple query with analyzer and pure parallel replicas SELECT number FROM join_inner_table__fuzz_146_replicated @@ -24,7 +26,7 @@ FROM join_inner_table__fuzz_146_replicated enable_analyzer = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', - allow_experimental_parallel_reading_from_replicas = 1; + enable_parallel_replicas = 1; SYSTEM FLUSH LOGS; -- There should be 2 different queries @@ -49,3 +51,5 @@ WHERE ) GROUP BY is_initial_query, query ORDER BY is_initial_query DESC, c, query; + +DROP TABLE join_inner_table__fuzz_146_replicated; diff --git a/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh index bf7170fd7fb4..47be7c2d372b 100755 --- a/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh +++ b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh @@ -33,7 +33,7 @@ function run_query_with_pure_parallel_replicas () { --max_parallel_replicas 3 \ --prefer_localhost_replica 1 \ --cluster_for_parallel_replicas 'test_cluster_one_shard_three_replicas_localhost' \ - --allow_experimental_parallel_reading_from_replicas 1 \ + --enable_parallel_replicas 1 \ --enable_analyzer 0 $CLICKHOUSE_CLIENT \ @@ -42,7 +42,7 @@ function run_query_with_pure_parallel_replicas () { --max_parallel_replicas 3 \ --prefer_localhost_replica 1 \ --cluster_for_parallel_replicas 'test_cluster_one_shard_three_replicas_localhost' \ - --allow_experimental_parallel_reading_from_replicas 1 \ + --enable_parallel_replicas 1 \ --enable_analyzer 1 } @@ -56,7 +56,7 @@ function run_query_with_custom_key_parallel_replicas () { --query "$2" \ --query_id "${1}_custom_key" \ --max_parallel_replicas 3 \ - --parallel_replicas_custom_key_filter_type 'default' \ + --parallel_replicas_mode 'custom_key_sampling' \ --parallel_replicas_custom_key "$2" \ --enable_analyzer 0 @@ -64,7 +64,7 @@ function run_query_with_custom_key_parallel_replicas () { --query "$2" \ --query_id "${1}_custom_key_analyzer" \ --max_parallel_replicas 3 \ - --parallel_replicas_custom_key_filter_type 'default' \ + --parallel_replicas_mode 'custom_key_sampling' \ --parallel_replicas_custom_key "$2" \ --enable_analyzer 1 } diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index 8a3b34e5cfa2..e0aeb39972be 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -6,16 +6,17 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"} function were_parallel_replicas_used () { - # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it $CLICKHOUSE_CLIENT --query " SELECT initial_query_id, - concat('Used parallel replicas: ', (countIf(initial_query_id != query_id) != 0)::bool::String) as used + concat('Used parallel replicas: ', (ProfileEvents['ParallelReplicasUsedCount'] > 0)::bool::String) as used FROM system.query_log WHERE event_date >= yesterday() AND initial_query_id LIKE '$1%' - GROUP BY initial_query_id - ORDER BY min(event_time_microseconds) ASC + AND query_id = initial_query_id + AND type = 'QueryFinish' + AND current_database = '$CLICKHOUSE_DATABASE' + ORDER BY event_time_microseconds ASC FORMAT TSV" } @@ -48,11 +49,11 @@ function run_query_with_pure_parallel_replicas () { --query "$3" \ --query_id "${1}_pure" \ --max_parallel_replicas 3 \ - --prefer_localhost_replica 1 \ --cluster_for_parallel_replicas "parallel_replicas" \ - --allow_experimental_parallel_reading_from_replicas 1 \ + --enable_parallel_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica "$2" \ + --max_threads 5 \ |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' } diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference index 521e3e2edbc2..595b35db610c 100644 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference @@ -4,8 +4,8 @@ 02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 10 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 3 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 33 estimated parallel replicas -02784_automatic_parallel_replicas_join-default_simple_join_0_pure Distinct parallel subqueries: 2 Used parallel replicas: true -02784_automatic_parallel_replicas_join-default_simple_join_10M_pure Distinct parallel subqueries: 0 Used parallel replicas: false -02784_automatic_parallel_replicas_join-default_simple_join_5M_pure Distinct parallel subqueries: 1 Used parallel replicas: true -02784_automatic_parallel_replicas_join-default_simple_join_1M_pure Distinct parallel subqueries: 1 Used parallel replicas: true -02784_automatic_parallel_replicas_join-default_simple_join_300k_pure Distinct parallel subqueries: 2 Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_0_pure Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_10M_pure Used parallel replicas: false +02784_automatic_parallel_replicas_join-default_simple_join_5M_pure Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_1M_pure Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_300k_pure Used parallel replicas: true diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh index 801cd22b79f6..61075de0f689 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh @@ -9,17 +9,17 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"} function were_parallel_replicas_used () { - # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it $CLICKHOUSE_CLIENT --query " SELECT initial_query_id, - concat('Distinct parallel subqueries: ' , countDistinctIf(query, initial_query_id != query_id)::String) as subqueries_parallelized, - concat('Used parallel replicas: ', (countIf(initial_query_id != query_id) != 0)::bool::String) as used + concat('Used parallel replicas: ', (ProfileEvents['ParallelReplicasUsedCount'] > 0)::bool::String) as used FROM system.query_log WHERE event_date >= yesterday() AND initial_query_id LIKE '$1%' - GROUP BY initial_query_id - ORDER BY min(event_time_microseconds) ASC + AND query_id = initial_query_id + AND type = 'QueryFinish' + AND current_database = '$CLICKHOUSE_DATABASE' + ORDER BY event_time_microseconds ASC FORMAT TSV" } @@ -67,7 +67,7 @@ function run_query_with_pure_parallel_replicas () { --prefer_localhost_replica 1 \ --parallel_replicas_prefer_local_join 0 \ --cluster_for_parallel_replicas "parallel_replicas" \ - --allow_experimental_parallel_reading_from_replicas 1 \ + --enable_parallel_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica "$2" \ |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' | sort -n -k2 -b | grep -Pv "\t0 estimated parallel replicas" diff --git a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql index 294c1325ba63..b68faa8500cb 100644 --- a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql +++ b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql @@ -10,7 +10,8 @@ SELECT count() FROM users PREWHERE uid > 2000; -- enable parallel replicas but with high rows threshold SET -allow_experimental_parallel_reading_from_replicas=1, +skip_unavailable_shards=1, +enable_parallel_replicas=1, max_parallel_replicas=3, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1, diff --git a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql index 1e6f9304c0cb..d5489ce657d1 100644 --- a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql +++ b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql @@ -14,13 +14,13 @@ insert into test select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; insert into test select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; -- 2 shards @@ -38,10 +38,10 @@ insert into test2 select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test2_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; insert into test2 select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test2_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; diff --git a/tests/queries/0_stateless/02841_parallel_replicas_summary.sh b/tests/queries/0_stateless/02841_parallel_replicas_summary.sh index c82d2c8b0c05..928d73a4a70b 100755 --- a/tests/queries/0_stateless/02841_parallel_replicas_summary.sh +++ b/tests/queries/0_stateless/02841_parallel_replicas_summary.sh @@ -27,6 +27,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE replicas_summary (n Int64) ENGINE = Mer query_id_base="02841_summary_$CLICKHOUSE_DATABASE" +# TODO: rethink the test, for now temporary disable parallel_replicas_local_plan echo " SELECT * FROM replicas_summary @@ -34,9 +35,10 @@ echo " SETTINGS max_parallel_replicas = 2, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', - allow_experimental_parallel_reading_from_replicas = 2, + enable_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, - interactive_delay=0 + interactive_delay=0, + parallel_replicas_local_plan=0 "\ | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_0" --data-binary @- -vvv 2>&1 \ | grep "Summary" | grep -cv '"read_rows":"0"' @@ -48,9 +50,10 @@ echo " SETTINGS max_parallel_replicas = 2, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', - allow_experimental_parallel_reading_from_replicas = 2, + enable_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, - interactive_delay=99999999999 + interactive_delay=99999999999, + parallel_replicas_local_plan=0 "\ | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_high" --data-binary @- -vvv 2>&1 \ | grep "Summary" | grep -cv '"read_rows":"0"' diff --git a/tests/queries/0_stateless/02861_index_set_incorrect_args.sql b/tests/queries/0_stateless/02861_index_set_incorrect_args.sql index 17b505cd0516..985e2a17ff0b 100644 --- a/tests/queries/0_stateless/02861_index_set_incorrect_args.sql +++ b/tests/queries/0_stateless/02861_index_set_incorrect_args.sql @@ -2,5 +2,5 @@ DROP TABLE IF EXISTS set_index__fuzz_41; CREATE TABLE set_index__fuzz_41 (`a` Date, `b` Nullable(DateTime64(3)), INDEX b_set b TYPE set(0) GRANULARITY 1) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO set_index__fuzz_41 (a) VALUES (today()); -SELECT b FROM set_index__fuzz_41 WHERE and(b = 256) SETTINGS force_data_skipping_indices = 'b_set', optimize_move_to_prewhere = 0, max_parallel_replicas=2, parallel_replicas_for_non_replicated_merge_tree=1, allow_experimental_parallel_reading_from_replicas=2; -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } +SELECT b FROM set_index__fuzz_41 WHERE and(b = 256) SETTINGS force_data_skipping_indices = 'b_set', optimize_move_to_prewhere = 0, max_parallel_replicas=2, parallel_replicas_for_non_replicated_merge_tree=1, enable_parallel_replicas=2; -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } DROP TABLE set_index__fuzz_41; diff --git a/tests/queries/0_stateless/02869_parallel_replicas_read_from_several.sql b/tests/queries/0_stateless/02869_parallel_replicas_read_from_several.sql index e040fae1fa6b..c4359f059ba3 100644 --- a/tests/queries/0_stateless/02869_parallel_replicas_read_from_several.sql +++ b/tests/queries/0_stateless/02869_parallel_replicas_read_from_several.sql @@ -24,5 +24,5 @@ system sync replica t3; SELECT count(), min(k), max(k), avg(k) FROM t1 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, +SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost', parallel_replicas_single_task_marks_count_multiplier = 0.001; diff --git a/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql index f59d38ceb046..51301fe4ea65 100644 --- a/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql +++ b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql @@ -2,12 +2,12 @@ DROP TABLE IF EXISTS tt; CREATE TABLE tt (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); INSERT INTO tt SELECT * FROM numbers(10); -SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, parallel_replicas_for_non_replicated_merge_tree=1; +SET enable_parallel_replicas=1, max_parallel_replicas=3, parallel_replicas_for_non_replicated_merge_tree=1; SELECT count() FROM clusterAllReplicas('test_cluster_two_shard_three_replicas_localhost', currentDatabase(), tt) settings log_comment='02875_190aed82-2423-413b-ad4c-24dcca50f65b'; SYSTEM FLUSH LOGS; -SET allow_experimental_parallel_reading_from_replicas=0; +SET enable_parallel_replicas=0; SELECT count() > 0 FROM system.text_log WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment = '02875_190aed82-2423-413b-ad4c-24dcca50f65b') AND message LIKE '%Parallel reading from replicas is disabled for cluster%'; diff --git a/tests/queries/0_stateless/02875_parallel_replicas_remote.sql b/tests/queries/0_stateless/02875_parallel_replicas_remote.sql index 5fbaf34b621e..259ed02b2a3a 100644 --- a/tests/queries/0_stateless/02875_parallel_replicas_remote.sql +++ b/tests/queries/0_stateless/02875_parallel_replicas_remote.sql @@ -2,12 +2,12 @@ DROP TABLE IF EXISTS tt; CREATE TABLE tt (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); INSERT INTO tt SELECT * FROM numbers(10); -SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, parallel_replicas_for_non_replicated_merge_tree=1; +SET enable_parallel_replicas=1, max_parallel_replicas=3, parallel_replicas_for_non_replicated_merge_tree=1; SELECT count() FROM remote('127.0.0.{1..6}', currentDatabase(), tt) settings log_comment='02875_89f3c39b-1919-48cb-b66e-ef9904e73146'; SYSTEM FLUSH LOGS; -SET allow_experimental_parallel_reading_from_replicas=0; +SET enable_parallel_replicas=0; SELECT count() > 0 FROM system.text_log WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment = '02875_89f3c39b-1919-48cb-b66e-ef9904e73146') AND message LIKE '%Parallel reading from replicas is disabled for cluster%'; diff --git a/tests/queries/0_stateless/02898_parallel_replicas_custom_key_final.sql b/tests/queries/0_stateless/02898_parallel_replicas_custom_key_final.sql index 7df88988bab1..32e2592430d3 100644 --- a/tests/queries/0_stateless/02898_parallel_replicas_custom_key_final.sql +++ b/tests/queries/0_stateless/02898_parallel_replicas_custom_key_final.sql @@ -8,12 +8,12 @@ SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02898_parallel_replicas_final) FINAL GROUP BY y ORDER BY y -SETTINGS max_parallel_replicas=3, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_custom_key_filter_type='default'; +SETTINGS max_parallel_replicas=3, enable_parallel_replicas=1, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_mode='custom_key_sampling'; SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02898_parallel_replicas_final) FINAL GROUP BY y ORDER BY y -SETTINGS max_parallel_replicas=3, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_custom_key_filter_type='range'; +SETTINGS max_parallel_replicas=3, enable_parallel_replicas=1, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_mode='custom_key_range'; -DROP TABLE 02898_parallel_replicas_final; \ No newline at end of file +DROP TABLE 02898_parallel_replicas_final; diff --git a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql index d8bfec12b3a8..cbb4ff5334c8 100644 --- a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql +++ b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql @@ -14,7 +14,7 @@ system sync replica t1; system sync replica t2; system sync replica t3; -SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; +SET enable_parallel_replicas=1, max_parallel_replicas=3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; -- default coordinator SELECT count(), min(k), max(k), avg(k) FROM t1 SETTINGS log_comment='02898_default_190aed82-2423-413b-ad4c-24dcca50f65b'; @@ -23,15 +23,16 @@ SELECT count(), min(k), max(k), avg(k) FROM t1 SETTINGS log_comment='02898_defau SYSTEM FLUSH LOGS; SELECT count() > 0 FROM system.text_log WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment='02898_default_190aed82-2423-413b-ad4c-24dcca50f65b') - AND message LIKE '%Total rows to read: 3000%' SETTINGS allow_experimental_parallel_reading_from_replicas=0; + AND message LIKE '%Total rows to read: 3000%' SETTINGS enable_parallel_replicas=0; -- reading in order coordinator -SELECT k, sipHash64(v) FROM t1 order by k limit 5 offset 998 SETTINGS optimize_read_in_order=1, log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b'; +-- disable parallel_replicas_local_plan since the test relay on traces which only present in case of no local plan +SELECT k, sipHash64(v) FROM t1 order by k limit 5 offset 998 SETTINGS optimize_read_in_order=1, parallel_replicas_local_plan=0, log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b'; SYSTEM FLUSH LOGS; SELECT count() > 0 FROM system.text_log WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b') - AND message LIKE '%Updated total rows to read: added % rows, total 3000 rows%' SETTINGS allow_experimental_parallel_reading_from_replicas=0; + AND message LIKE '%Updated total rows to read: added % rows, total 3000 rows%' SETTINGS enable_parallel_replicas=0; DROP TABLE t1 SYNC; DROP TABLE t2 SYNC; diff --git a/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh b/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh index def813c17b47..2fc4502c5314 100755 --- a/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh +++ b/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh @@ -30,7 +30,7 @@ $CLICKHOUSE_CLIENT \ --max_parallel_replicas 3 \ --prefer_localhost_replica 1 \ --cluster_for_parallel_replicas "test_cluster_one_shard_three_replicas_localhost" \ - --allow_experimental_parallel_reading_from_replicas 1 \ + --enable_parallel_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica 0 \ --query " @@ -68,7 +68,7 @@ $CLICKHOUSE_CLIENT \ --max_parallel_replicas 3 \ --prefer_localhost_replica 1 \ --cluster_for_parallel_replicas "test_cluster_one_shard_three_replicas_localhost" \ - --allow_experimental_parallel_reading_from_replicas 1 \ + --enable_parallel_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica 0 \ --query "SELECT * FROM (SELECT year, month, day, count(*) FROM days GROUP BY year, month, day WITH ROLLUP) ORDER BY 1, 2, 3"; diff --git a/tests/queries/0_stateless/02918_parallel_replicas_custom_key_unavailable_replica.reference b/tests/queries/0_stateless/02918_parallel_replicas_custom_key_unavailable_replica.reference index 2d97dd0e12e7..490d36aafdf8 100644 --- a/tests/queries/0_stateless/02918_parallel_replicas_custom_key_unavailable_replica.reference +++ b/tests/queries/0_stateless/02918_parallel_replicas_custom_key_unavailable_replica.reference @@ -3,7 +3,7 @@ SELECT y, count() FROM cluster(test_cluster_1_shard_3_replicas_1_unavailable, currentDatabase(), 02918_parallel_replicas) GROUP BY y ORDER BY y -SETTINGS max_parallel_replicas=3, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_custom_key_filter_type='default'; +SETTINGS max_parallel_replicas=3, enable_parallel_replicas=1, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_mode='custom_key_sampling'; 0 250 1 250 2 250 @@ -12,7 +12,7 @@ SELECT y, count() FROM cluster(test_cluster_1_shard_3_replicas_1_unavailable, currentDatabase(), 02918_parallel_replicas) GROUP BY y ORDER BY y -SETTINGS max_parallel_replicas=3, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_custom_key_filter_type='range'; +SETTINGS max_parallel_replicas=3, enable_parallel_replicas=1, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_mode='custom_key_range'; 0 250 1 250 2 250 @@ -22,7 +22,7 @@ SELECT y, count() FROM cluster(test_cluster_1_shard_3_replicas_1_unavailable, currentDatabase(), 02918_parallel_replicas) GROUP BY y ORDER BY y -SETTINGS max_parallel_replicas=3, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_custom_key_filter_type='default'; +SETTINGS max_parallel_replicas=3, enable_parallel_replicas=1, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_mode='custom_key_sampling'; 0 250 1 250 2 250 diff --git a/tests/queries/0_stateless/02918_parallel_replicas_custom_key_unavailable_replica.sql b/tests/queries/0_stateless/02918_parallel_replicas_custom_key_unavailable_replica.sql index c36b5bebd582..763a4530b944 100644 --- a/tests/queries/0_stateless/02918_parallel_replicas_custom_key_unavailable_replica.sql +++ b/tests/queries/0_stateless/02918_parallel_replicas_custom_key_unavailable_replica.sql @@ -14,20 +14,20 @@ SELECT y, count() FROM cluster(test_cluster_1_shard_3_replicas_1_unavailable, currentDatabase(), 02918_parallel_replicas) GROUP BY y ORDER BY y -SETTINGS max_parallel_replicas=3, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_custom_key_filter_type='default'; +SETTINGS max_parallel_replicas=3, enable_parallel_replicas=1, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_mode='custom_key_sampling'; SELECT y, count() FROM cluster(test_cluster_1_shard_3_replicas_1_unavailable, currentDatabase(), 02918_parallel_replicas) GROUP BY y ORDER BY y -SETTINGS max_parallel_replicas=3, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_custom_key_filter_type='range'; +SETTINGS max_parallel_replicas=3, enable_parallel_replicas=1, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_mode='custom_key_range'; SET use_hedged_requests=0; SELECT y, count() FROM cluster(test_cluster_1_shard_3_replicas_1_unavailable, currentDatabase(), 02918_parallel_replicas) GROUP BY y ORDER BY y -SETTINGS max_parallel_replicas=3, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_custom_key_filter_type='default'; +SETTINGS max_parallel_replicas=3, enable_parallel_replicas=1, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_mode='custom_key_sampling'; -- { echoOff } SET send_logs_level='warning'; diff --git a/tests/queries/0_stateless/02922_analyzer_aggregate_nothing_type.sql b/tests/queries/0_stateless/02922_analyzer_aggregate_nothing_type.sql index 5b9343c6e13b..c855e01a2b3a 100644 --- a/tests/queries/0_stateless/02922_analyzer_aggregate_nothing_type.sql +++ b/tests/queries/0_stateless/02922_analyzer_aggregate_nothing_type.sql @@ -88,7 +88,7 @@ CREATE TABLE t1 (`n` UInt64) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO t1 SELECT * FROM numbers(10); SET - allow_experimental_parallel_reading_from_replicas=1, +enable_parallel_replicas=1, max_parallel_replicas=2, use_hedged_requests=0, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost', diff --git a/tests/queries/0_stateless/02932_parallel_replicas_fuzzer.sql b/tests/queries/0_stateless/02932_parallel_replicas_fuzzer.sql index e5cbe100a58b..038f5c1c98a3 100644 --- a/tests/queries/0_stateless/02932_parallel_replicas_fuzzer.sql +++ b/tests/queries/0_stateless/02932_parallel_replicas_fuzzer.sql @@ -9,7 +9,7 @@ SELECT key, value1, value2, toUInt64(min(time)) AS start_ts FROM join_inner_tabl max_parallel_replicas = 3, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', - allow_experimental_parallel_reading_from_replicas = 1, + enable_parallel_replicas = 1, use_hedged_requests = 0; @@ -22,7 +22,7 @@ ORDER BY nan DESC, [0, NULL, NULL, NULL, NULL] DESC FORMAT Null SETTINGS max_parallel_replicas = 3, - allow_experimental_parallel_reading_from_replicas = 1, + enable_parallel_replicas = 1, use_hedged_requests = 0, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; @@ -35,4 +35,4 @@ ORDER BY nan DESC, _CAST([0, NULL, NULL, NULL, NULL], 'Array(Nullable(UInt8))') DESC FORMAT Null -SETTINGS receive_timeout = 10., receive_data_timeout_ms = 10000, use_hedged_requests = 0, allow_suspicious_low_cardinality_types = 1, max_parallel_replicas = 3, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, log_queries = 1, table_function_remote_max_addresses = 200, enable_analyzer = 1; +SETTINGS receive_timeout = 10., receive_data_timeout_ms = 10000, use_hedged_requests = 0, allow_suspicious_low_cardinality_types = 1, max_parallel_replicas = 3, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_parallel_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, log_queries = 1, table_function_remote_max_addresses = 200, enable_analyzer = 1; diff --git a/tests/queries/0_stateless/02935_parallel_replicas_settings.sql b/tests/queries/0_stateless/02935_parallel_replicas_settings.sql index be6f1c2958cc..5e3b1a13f1ac 100644 --- a/tests/queries/0_stateless/02935_parallel_replicas_settings.sql +++ b/tests/queries/0_stateless/02935_parallel_replicas_settings.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS test_parallel_replicas_settings; CREATE TABLE test_parallel_replicas_settings (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); INSERT INTO test_parallel_replicas_settings SELECT * FROM numbers(10); -SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=3, parallel_replicas_for_non_replicated_merge_tree=1; +SET enable_parallel_replicas=2, max_parallel_replicas=3, parallel_replicas_for_non_replicated_merge_tree=1; SET cluster_for_parallel_replicas=''; SELECT count() FROM test_parallel_replicas_settings WHERE NOT ignore(*); -- { serverError CLUSTER_DOESNT_EXIST } @@ -16,20 +16,20 @@ SELECT count() > 0 FROM system.text_log WHERE yesterday() <= event_date AND query_id in (select query_id from system.query_log where current_database=currentDatabase() AND log_comment='0_f621c4f2-4da7-4a7c-bb6d-052c442d0f7f') AND level = 'Information' - AND message ILIKE '%Disabling ''use_hedged_requests'' in favor of ''allow_experimental_parallel_reading_from_replicas''%' -SETTINGS allow_experimental_parallel_reading_from_replicas=0; + AND message ILIKE '%Disabling ''use_hedged_requests'' in favor of ''enable_parallel_replicas''%' +SETTINGS enable_parallel_replicas=0; SET use_hedged_requests=1; SELECT count() FROM test_parallel_replicas_settings WHERE NOT ignore(*) settings log_comment='1_f621c4f2-4da7-4a7c-bb6d-052c442d0f7f'; SYSTEM FLUSH LOGS; -SET allow_experimental_parallel_reading_from_replicas=0; +SET enable_parallel_replicas=0; SELECT count() > 0 FROM system.text_log WHERE yesterday() <= event_date AND query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment = '1_f621c4f2-4da7-4a7c-bb6d-052c442d0f7f') AND level = 'Warning' - AND message ILIKE '%Setting ''use_hedged_requests'' explicitly with enabled ''allow_experimental_parallel_reading_from_replicas'' has no effect%' -SETTINGS allow_experimental_parallel_reading_from_replicas=0; + AND message ILIKE '%Setting ''use_hedged_requests'' explicitly with enabled ''enable_parallel_replicas'' has no effect%' +SETTINGS enable_parallel_replicas=0; DROP TABLE test_parallel_replicas_settings; diff --git a/tests/queries/0_stateless/02946_parallel_replicas_distributed.sql b/tests/queries/0_stateless/02946_parallel_replicas_distributed.sql index 1afd4ff01929..e6de0e8cee64 100644 --- a/tests/queries/0_stateless/02946_parallel_replicas_distributed.sql +++ b/tests/queries/0_stateless/02946_parallel_replicas_distributed.sql @@ -11,7 +11,7 @@ ENGINE = Distributed(test_cluster_one_shard_three_replicas_localhost, currentDat SELECT count(), sum(id) FROM test_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; DROP TABLE test_d; DROP TABLE test; diff --git a/tests/queries/0_stateless/02946_parallel_replicas_force_primary_key.sql b/tests/queries/0_stateless/02946_parallel_replicas_force_primary_key.sql index d33c8cdbc933..b307932412a0 100644 --- a/tests/queries/0_stateless/02946_parallel_replicas_force_primary_key.sql +++ b/tests/queries/0_stateless/02946_parallel_replicas_force_primary_key.sql @@ -22,10 +22,10 @@ FROM t1 WHERE k > 0 GROUP BY k ORDER BY k -SETTINGS force_primary_key = 1, allow_experimental_parallel_reading_from_replicas = 0; +SETTINGS force_primary_key = 1, enable_parallel_replicas = 0; -- parallel replicas, primary key is used -SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; +SET enable_parallel_replicas=1, max_parallel_replicas=3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; SELECT k, count() diff --git a/tests/queries/0_stateless/02947_parallel_replicas_remote.sql b/tests/queries/0_stateless/02947_parallel_replicas_remote.sql index 345d9f9cb03f..2f8d777ab368 100644 --- a/tests/queries/0_stateless/02947_parallel_replicas_remote.sql +++ b/tests/queries/0_stateless/02947_parallel_replicas_remote.sql @@ -7,6 +7,6 @@ AS select *, '2023-12-25' from numbers(100); SELECT count(), sum(id) FROM remote('127.0.0.1|127.0.0.2|127.0.0.3|127.0.0.4', currentDatabase(), test) -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 4, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree = 1; -- { serverError CLUSTER_DOESNT_EXIST } +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 4, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree = 1; -- { serverError CLUSTER_DOESNT_EXIST } DROP TABLE test; diff --git a/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.sql b/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.sql index ab6e1532299c..1e71d2feba04 100644 --- a/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.sql +++ b/tests/queries/0_stateless/02949_parallel_replicas_in_subquery.sql @@ -8,23 +8,23 @@ INSERT INTO merge_tree_in_subqueries VALUES(5, 'test5', 0); SET max_parallel_replicas=3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost', parallel_replicas_for_non_replicated_merge_tree=1; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=0; -- { serverError SUPPORT_IS_DISABLED } -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=1; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS allow_experimental_parallel_reading_from_replicas=1; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS enable_parallel_replicas=2, parallel_replicas_allow_in_with_subquery=0; -- { serverError SUPPORT_IS_DISABLED } +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS enable_parallel_replicas=2, parallel_replicas_allow_in_with_subquery=1; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0) SETTINGS enable_parallel_replicas=1; SELECT '---'; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=0; -- { serverError SUPPORT_IS_DISABLED }; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=1; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=1; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS enable_parallel_replicas=2, parallel_replicas_allow_in_with_subquery=0; -- { serverError SUPPORT_IS_DISABLED }; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS enable_parallel_replicas=2, parallel_replicas_allow_in_with_subquery=1; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id SETTINGS enable_parallel_replicas=1; SELECT '---'; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=0; -- { serverError SUPPORT_IS_DISABLED }; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=1; -SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS allow_experimental_parallel_reading_from_replicas=1; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS enable_parallel_replicas=2, parallel_replicas_allow_in_with_subquery=0; -- { serverError SUPPORT_IS_DISABLED }; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS enable_parallel_replicas=2, parallel_replicas_allow_in_with_subquery=1; +SELECT * FROM merge_tree_in_subqueries WHERE id IN (SELECT 1) ORDER BY id SETTINGS enable_parallel_replicas=1; -- IN with tuples is allowed SELECT '---'; -SELECT id, name FROM merge_tree_in_subqueries WHERE (id, name) IN (3, 'test3') SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=0; -SELECT id, name FROM merge_tree_in_subqueries WHERE (id, name) IN (3, 'test3') SETTINGS allow_experimental_parallel_reading_from_replicas=2, parallel_replicas_allow_in_with_subquery=1; +SELECT id, name FROM merge_tree_in_subqueries WHERE (id, name) IN (3, 'test3') SETTINGS enable_parallel_replicas=2, parallel_replicas_allow_in_with_subquery=0; +SELECT id, name FROM merge_tree_in_subqueries WHERE (id, name) IN (3, 'test3') SETTINGS enable_parallel_replicas=2, parallel_replicas_allow_in_with_subquery=1; DROP TABLE IF EXISTS merge_tree_in_subqueries; diff --git a/tests/queries/0_stateless/02949_parallel_replicas_scalar_subquery_big_integer.sql b/tests/queries/0_stateless/02949_parallel_replicas_scalar_subquery_big_integer.sql index 26f87180ab21..3b5f6277df7c 100644 --- a/tests/queries/0_stateless/02949_parallel_replicas_scalar_subquery_big_integer.sql +++ b/tests/queries/0_stateless/02949_parallel_replicas_scalar_subquery_big_integer.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (x UInt8) ENGINE = MergeTree ORDER BY x; INSERT INTO test VALUES (1), (2), (3); -SET allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree = 1; +SET enable_parallel_replicas = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree = 1; WITH (SELECT '111111111111111111111111111111111111111'::UInt128) AS v SELECT sum(x), max(v) FROM test; diff --git a/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference b/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference index 21b7b527b7a5..c1265b7ca145 100644 --- a/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference +++ b/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference @@ -1,4 +1,4 @@ -100 4950 +10000 49995000 1 89 90 @@ -6,3 +6,9 @@ 92 93 1 +93 +92 +91 +90 +89 +1 diff --git a/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql b/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql index 22f55acd3652..b9295dbc2317 100644 --- a/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql +++ b/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql @@ -2,24 +2,30 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (k UInt64, v String) ENGINE = MergeTree -ORDER BY k; +ORDER BY k +SETTINGS index_granularity=1; -INSERT INTO test SELECT number, toString(number) FROM numbers(100); - -SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; +INSERT INTO test SELECT number, toString(number) FROM numbers(10_000); +SET enable_parallel_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; -- default coordinator SELECT count(), sum(k) FROM test SETTINGS log_comment = '02950_parallel_replicas_used_replicas_count'; SYSTEM FLUSH LOGS; -SELECT ProfileEvents['ParallelReplicasUsedCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; +SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS enable_parallel_replicas=0; -- In order coordinator -SELECT k FROM test order by k limit 5 offset 89 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_2'; +SELECT k FROM test order by k limit 5 offset 89 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_2', merge_tree_min_rows_for_concurrent_read=1, max_threads=1; + +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_2' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS enable_parallel_replicas=0; + +-- In reverse order coordinator +SELECT k FROM test order by k desc limit 5 offset 9906 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_3', merge_tree_min_rows_for_concurrent_read=1, max_threads=1; SYSTEM FLUSH LOGS; -SELECT ProfileEvents['ParallelReplicasUsedCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_2' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; +SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_3' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS enable_parallel_replicas=0; DROP TABLE test; diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.reference b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.reference index c0485b817c4e..87e78d7d1169 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.reference +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.reference @@ -11,10 +11,8 @@ nested join with analyzer and parallel replicas, both global 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) DefaultCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) DefaultCoordinator: Coordination done nested join with analyzer and parallel replicas, global + local @@ -29,10 +27,8 @@ nested join with analyzer and parallel replicas, global + local 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table3`) AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table3`) AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) DefaultCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) DefaultCoordinator: Coordination done nested join with analyzer and parallel replicas, both local, both full sorting merge join @@ -47,10 +43,8 @@ nested join with analyzer and parallel replicas, both local, both full sorting m 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) WithOrderCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) WithOrderCoordinator: Coordination done nested join with analyzer and parallel replicas, both local, both full sorting and hash join @@ -65,10 +59,8 @@ nested join with analyzer and parallel replicas, both local, both full sorting a 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) DefaultCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) WithOrderCoordinator: Coordination done nested join with analyzer and parallel replicas, both local, both full sorting and hash join @@ -83,8 +75,6 @@ nested join with analyzer and parallel replicas, both local, both full sorting a 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) WithOrderCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) DefaultCoordinator: Coordination done diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh index e954cb0e78e0..6c296257c538 100755 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh @@ -17,6 +17,8 @@ insert into num_1 select number * 2, toString(number * 2) from numbers(1e7); insert into num_2 select number * 3, -number from numbers(1.5e6); " +PARALLEL_REPLICAS_SETTINGS="enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_local_plan=1" + ############## echo echo "nested join with analyzer and parallel replicas, both global" @@ -26,18 +28,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, send_logs_level='trace', parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -51,18 +49,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, send_logs_level='trace', parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -77,18 +71,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -102,18 +92,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -127,18 +113,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='hash'" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" 2>&1 | +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='hash'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference index 36f02b2f764c..93003b6cf6d8 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference @@ -2,7 +2,7 @@ set parallel_replicas_prefer_local_join = 0; -- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode. -select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -18,9 +18,21 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x 13 13 0 0 0 0 14 14 14 14 0 0 15 15 0 0 0 0 -explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -Expression - ReadFromRemoteParallelReplicas +explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +Union + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + ReadFromMemoryStorage + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas -- -- The same query with cte; with sub1 as (select x, y from tab1 where x != 2), @@ -29,7 +41,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -52,11 +64,25 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Sorting + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + ReadFromMemoryStorage + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas -- -- GROUP BY should work up to WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -65,7 +91,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) select * from sub5 order by key -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 54 54 50 50 12 12 0 64 64 0 0 0 0 1 explain description=0 @@ -75,13 +101,27 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) select * from sub5 order by key -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression MergingAggregated - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Aggregating + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + ReadFromMemoryStorage + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas -- -- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -90,7 +130,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -113,15 +153,27 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression Join - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- ORDER BY in sub1 : sub1 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2 order by y), @@ -130,7 +182,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -153,19 +205,31 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression Join Expression Join + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub3: sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -174,7 +238,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 6 6 6 6 0 0 8 8 8 8 0 0 @@ -197,17 +261,29 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Join Expression Join + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub5: sub5 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -215,7 +291,7 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) -select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +select * from sub5 order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 @@ -237,19 +313,31 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) -select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +select * from sub5 order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression Join - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas Expression Join - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- Subqueries for IN allowed with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -258,7 +346,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -281,11 +369,31 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Sorting + Expression + Join + Expression + Join + Expression + CreatingSets + Expression + Expression + ReadFromMergeTree + CreatingSet + Expression + Filter + ReadFromSystemNumbers + Expression + ReadFromMemoryStorage + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas -- -- Subqueries for IN are not allowed with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -294,7 +402,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; +SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -317,7 +425,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0;-- { echoOn } +SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0;-- { echoOn } Expression Sorting Expression @@ -333,13 +441,21 @@ Expression Expression Filter ReadFromSystemNumbers + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas set parallel_replicas_prefer_local_join = 1; -- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode. -select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -355,9 +471,23 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x 13 13 0 0 0 0 14 14 14 14 0 0 15 15 0 0 0 0 -explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -Expression - ReadFromRemoteParallelReplicas +explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +Union + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- The same query with cte; with sub1 as (select x, y from tab1 where x != 2), @@ -366,7 +496,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -389,11 +519,27 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Sorting + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- GROUP BY should work up to WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -402,7 +548,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) select * from sub5 order by key -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 54 54 50 50 12 12 0 64 64 0 0 0 0 1 explain description=0 @@ -412,13 +558,29 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) select * from sub5 order by key -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression MergingAggregated - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Aggregating + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -427,7 +589,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -450,15 +612,28 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression Join - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- ORDER BY in sub1 : sub1 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2 order by y), @@ -467,7 +642,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -490,19 +665,31 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression Join Expression Join + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub3: sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -511,7 +698,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 6 6 6 6 0 0 8 8 8 8 0 0 @@ -534,17 +721,29 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Join Expression Join + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub5: sub5 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -552,7 +751,7 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) -select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +select * from sub5 order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 @@ -574,19 +773,31 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) -select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +select * from sub5 order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression Join - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas Expression Join - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- Subqueries for IN allowed with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -595,7 +806,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -618,11 +829,33 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Sorting + Expression + Join + Expression + Join + Expression + CreatingSets + Expression + Expression + ReadFromMergeTree + CreatingSet + Expression + Filter + ReadFromSystemNumbers + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- Subqueries for IN are not allowed with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -631,7 +864,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; +SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -654,7 +887,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; +SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; Expression Sorting Expression @@ -670,7 +903,15 @@ Expression Expression Filter ReadFromSystemNumbers + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 index 775663768fef..23291881eb4b 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 @@ -10,6 +10,8 @@ insert into tab1 select number, number, number from numbers(16); insert into tab2 select number * 2, number * 2 from numbers(8); insert into tab3 select number * 4, number * 4 from numbers(4); +set parallel_replicas_local_plan=1; + {% for use_global_in in [0, 1] -%} -- { echoOn } @@ -17,8 +19,8 @@ insert into tab3 select number * 4, number * 4 from numbers(4); set parallel_replicas_prefer_local_join = {{use_global_in}}; -- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode. -select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- The same query with cte; with sub1 as (select x, y from tab1 where x != 2), @@ -27,7 +29,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x != 2), @@ -36,7 +38,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- GROUP BY should work up to WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -45,7 +47,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) select * from sub5 order by key -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x != 2), @@ -54,7 +56,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) select * from sub5 order by key -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -63,7 +65,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x != 2), @@ -72,7 +74,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- ORDER BY in sub1 : sub1 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2 order by y), @@ -81,7 +83,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x != 2 order by y), @@ -90,7 +92,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- RIGHT JOIN in sub3: sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -99,7 +101,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x != 2), @@ -108,7 +110,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- RIGHT JOIN in sub5: sub5 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -116,7 +118,7 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) -select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +select * from sub5 order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x != 2), @@ -124,7 +126,7 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) -select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +select * from sub5 order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- Subqueries for IN allowed @@ -134,7 +136,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -143,7 +145,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- Subqueries for IN are not allowed @@ -153,7 +155,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; +SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; explain description=0 with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -162,6 +164,6 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; +SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; {%- endfor %} diff --git a/tests/queries/0_stateless/02972_parallel_replicas_cte.sql b/tests/queries/0_stateless/02972_parallel_replicas_cte.sql index 767cd61216b7..10eb1eb1d718 100644 --- a/tests/queries/0_stateless/02972_parallel_replicas_cte.sql +++ b/tests/queries/0_stateless/02972_parallel_replicas_cte.sql @@ -13,21 +13,21 @@ SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups. WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 100) SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; +SETTINGS enable_parallel_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- Testing that it is disabled for enable_analyzer=0. With analyzer it will be supported (with correct result) WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 100) SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a -SETTINGS enable_analyzer = 0, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED } +SETTINGS enable_analyzer = 0, enable_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED } --- Disabled for any value of allow_experimental_parallel_reading_from_replicas != 1, not just 2 +-- Disabled for any value of enable_parallel_replicas != 1, not just 2 WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 100) SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a -SETTINGS enable_analyzer = 0, allow_experimental_parallel_reading_from_replicas = 512, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED } +SETTINGS enable_analyzer = 0, enable_parallel_replicas = 512, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED } -- Sanitizer SELECT count() FROM pr_2 JOIN numbers(10) as pr_1 ON pr_2.a = pr_1.number -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; +SETTINGS enable_parallel_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- Parallel replicas detection should work inside subqueries SELECT * @@ -36,7 +36,7 @@ FROM WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 100) SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a ) -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; +SETTINGS enable_parallel_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- Subquery + subquery SELECT count() @@ -49,7 +49,7 @@ FROM SELECT count() as c FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a ) ) -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; +SETTINGS enable_parallel_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; CREATE TABLE numbers_1e3 ( @@ -74,7 +74,7 @@ WITH ) SELECT count() FROM cte2 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; +SETTINGS enable_parallel_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; DROP TABLE IF EXISTS numbers_1e6; DROP TABLE IF EXISTS pr_1; diff --git a/tests/queries/0_stateless/02981_insert_select_resize_to_max_insert_threads.sh b/tests/queries/0_stateless/02981_insert_select_resize_to_max_insert_threads.sh index 7ad5a2179f9a..77eddbcf63fc 100755 --- a/tests/queries/0_stateless/02981_insert_select_resize_to_max_insert_threads.sh +++ b/tests/queries/0_stateless/02981_insert_select_resize_to_max_insert_threads.sh @@ -49,7 +49,7 @@ ${CLICKHOUSE_CLIENT} --max_insert_threads "$max_insert_threads" --parallel_distr """ | grep -v EmptySink | grep -c Sink echo "inserting into a remote table from remote (reading with parallel replicas) with concurrency max_insert_threads" -${CLICKHOUSE_CLIENT} --max_insert_threads "$max_insert_threads" --allow_experimental_parallel_reading_from_replicas 2 --cluster_for_parallel_replicas 'parallel_replicas' --max_parallel_replicas 3 -q """ +${CLICKHOUSE_CLIENT} --max_insert_threads "$max_insert_threads" --enable_parallel_replicas 2 --cluster_for_parallel_replicas 'parallel_replicas' --max_parallel_replicas 3 -q """ EXPLAIN PIPELINE INSERT INTO t3_dist SELECT * FROM t4_pr; diff --git a/tests/queries/0_stateless/02982_parallel_replicas_unexpected_cluster.sql b/tests/queries/0_stateless/02982_parallel_replicas_unexpected_cluster.sql index 210b7d2a18a5..d70da5b5ae7f 100644 --- a/tests/queries/0_stateless/02982_parallel_replicas_unexpected_cluster.sql +++ b/tests/queries/0_stateless/02982_parallel_replicas_unexpected_cluster.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS test_unexpected_cluster; CREATE TABLE test_unexpected_cluster (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); INSERT INTO test_unexpected_cluster SELECT * FROM numbers(10); -SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=2, cluster_for_parallel_replicas='test_cluster_two_shards', parallel_replicas_for_non_replicated_merge_tree=1; +SET enable_parallel_replicas=2, max_parallel_replicas=2, cluster_for_parallel_replicas='test_cluster_two_shards', parallel_replicas_for_non_replicated_merge_tree=1; SELECT count() FROM test_unexpected_cluster WHERE NOT ignore(*); -- { serverError UNEXPECTED_CLUSTER } DROP TABLE test_unexpected_cluster; diff --git a/tests/queries/0_stateless/03006_parallel_replicas_cte_explain_syntax_crash.sql b/tests/queries/0_stateless/03006_parallel_replicas_cte_explain_syntax_crash.sql index 7c8d6dd9affe..ff56540b3916 100644 --- a/tests/queries/0_stateless/03006_parallel_replicas_cte_explain_syntax_crash.sql +++ b/tests/queries/0_stateless/03006_parallel_replicas_cte_explain_syntax_crash.sql @@ -21,7 +21,7 @@ AS SELECT * FROM numbers(10); SET enable_analyzer = 1; -SET allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=0; +SET enable_parallel_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=0; EXPLAIN SYNTAX WITH diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference index e69de29bb2d1..8b8d0e5d565c 100644 --- a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference +++ b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference @@ -0,0 +1 @@ +default url_na_log 1 130000 130 diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql index 4b84646c0345..a2198086b107 100644 --- a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql +++ b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql @@ -21,11 +21,21 @@ SELECT FROM numbers(130000) SETTINGS max_insert_block_size = 200000; +SET max_block_size = 1048576, max_threads = 1, enable_parallel_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=10000; + +EXPLAIN ESTIMATE +SELECT count() +FROM url_na_log +PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20') +SETTINGS parallel_replicas_local_plan=0; + +-- here parallel replicas uses local snapshot as working set +-- so, the estimation can be done EXPLAIN ESTIMATE SELECT count() FROM url_na_log PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20') -SETTINGS max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=10000; +SETTINGS allow_experimental_analyzer=1, parallel_replicas_local_plan=1; DROP POLICY url_na_log_policy0 ON url_na_log; DROP TABLE url_na_log; diff --git a/tests/queries/0_stateless/03093_reading_bug_with_parallel_replicas.sql b/tests/queries/0_stateless/03093_reading_bug_with_parallel_replicas.sql index 9592764baa60..00dd17c4904e 100644 --- a/tests/queries/0_stateless/03093_reading_bug_with_parallel_replicas.sql +++ b/tests/queries/0_stateless/03093_reading_bug_with_parallel_replicas.sql @@ -3,7 +3,7 @@ set max_threads = 16; set use_hedged_requests = 0; set max_parallel_replicas = 3; set cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; -set allow_experimental_parallel_reading_from_replicas = 1; +set enable_parallel_replicas = 1; set parallel_replicas_for_non_replicated_merge_tree = 1; set allow_aggregate_partitions_independently = 1; diff --git a/tests/queries/0_stateless/03143_parallel_replicas_mat_view_bug.sql b/tests/queries/0_stateless/03143_parallel_replicas_mat_view_bug.sql index 02a8a2f3ce07..10e1de815344 100644 --- a/tests/queries/0_stateless/03143_parallel_replicas_mat_view_bug.sql +++ b/tests/queries/0_stateless/03143_parallel_replicas_mat_view_bug.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS mv_table; DROP TABLE IF EXISTS null_table; -SET cluster_for_parallel_replicas='parallel_replicas', max_parallel_replicas=4, allow_experimental_parallel_reading_from_replicas=1; +SET cluster_for_parallel_replicas='parallel_replicas', max_parallel_replicas=4, enable_parallel_replicas=1; SET enable_analyzer=1; CREATE TABLE null_table (str String) ENGINE = Null; diff --git a/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql index b1018d000829..7fca1c556f86 100644 --- a/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql +++ b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql @@ -26,5 +26,4 @@ ENGINE = MergeTree ORDER BY (date, pull_request_number, commit_sha, check_name, insert into checks select * from generateRandom() limit 1; - -select trimLeft(explain) from (explain SELECT count(1) FROM checks WHERE test_name IS NOT NULL) where explain like '%ReadFromPreparedSource%' SETTINGS enable_analyzer = 1, allow_experimental_parallel_reading_from_replicas = 0; +select trimLeft(explain) from (explain SELECT count(1) FROM checks WHERE test_name IS NOT NULL) where explain like '%ReadFromPreparedSource%' SETTINGS enable_analyzer = 1, enable_parallel_replicas = 0; diff --git a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql index 58143395e444..5ed492def4d2 100644 --- a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql +++ b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql @@ -4,86 +4,126 @@ CREATE TABLE range_filter_custom_range_test (k UInt64) ENGINE=MergeTree ORDER BY INSERT INTO range_filter_custom_range_test SELECT number + 5 from numbers(10); -SELECT count() -FROM +SELECT count() +FROM ( SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) - SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, - parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', - parallel_replicas_custom_key_range_lower = 5, parallel_replicas_custom_key_range_upper = 15 + SETTINGS prefer_localhost_replica = 0, + max_parallel_replicas = 3, + distributed_group_by_no_merge = 0, + enable_parallel_replicas = 1, + parallel_replicas_custom_key = 'k', + parallel_replicas_mode = 'custom_key_range', + parallel_replicas_custom_key_range_lower = 5, + parallel_replicas_custom_key_range_upper = 15 ); -SELECT count() -FROM +SELECT count() +FROM ( SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) - SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, - parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', - parallel_replicas_custom_key_range_lower = 4, parallel_replicas_custom_key_range_upper = 14 + SETTINGS prefer_localhost_replica = 0, + max_parallel_replicas = 3, + distributed_group_by_no_merge = 0, + enable_parallel_replicas = 1, + parallel_replicas_custom_key = 'k', + parallel_replicas_mode = 'custom_key_range', + parallel_replicas_custom_key_range_lower = 4, + parallel_replicas_custom_key_range_upper = 14 ); -SELECT count() -FROM +SELECT count() +FROM ( SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) - SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, - parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', - parallel_replicas_custom_key_range_lower = 6, parallel_replicas_custom_key_range_upper = 17 + SETTINGS prefer_localhost_replica = 0, + max_parallel_replicas = 3, + distributed_group_by_no_merge = 0, + enable_parallel_replicas = 1, + parallel_replicas_custom_key = 'k', + parallel_replicas_mode = 'custom_key_range', + parallel_replicas_custom_key_range_lower = 6, + parallel_replicas_custom_key_range_upper = 17 ); -SELECT count() -FROM +SELECT count() +FROM ( SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) - SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, - parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', - parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 15 + SETTINGS prefer_localhost_replica = 0, + max_parallel_replicas = 3, + distributed_group_by_no_merge = 0, + enable_parallel_replicas = 1, + parallel_replicas_custom_key = 'k', + parallel_replicas_mode = 'custom_key_range', + parallel_replicas_custom_key_range_lower = 0, + parallel_replicas_custom_key_range_upper = 15 ); -SELECT count() -FROM +SELECT count() +FROM ( SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) - SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, - parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', - parallel_replicas_custom_key_range_lower = 15, parallel_replicas_custom_key_range_upper = 25 + SETTINGS prefer_localhost_replica = 0, + max_parallel_replicas = 3, + distributed_group_by_no_merge = 0, + enable_parallel_replicas = 1, + parallel_replicas_custom_key = 'k', + parallel_replicas_mode = 'custom_key_range', + parallel_replicas_custom_key_range_lower = 15, + parallel_replicas_custom_key_range_upper = 25 ); -SELECT count() -FROM +SELECT count() +FROM ( SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) - SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, - parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', - parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 5 + SETTINGS prefer_localhost_replica = 0, + max_parallel_replicas = 3, + distributed_group_by_no_merge = 0, + enable_parallel_replicas = 1, + parallel_replicas_custom_key = 'k', + parallel_replicas_mode = 'custom_key_range', + parallel_replicas_custom_key_range_lower = 0, + parallel_replicas_custom_key_range_upper = 5 ); -SELECT count() -FROM +SELECT count() +FROM ( SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) - SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, - parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', - parallel_replicas_custom_key_range_lower = 500, parallel_replicas_custom_key_range_upper = 10000 + SETTINGS prefer_localhost_replica = 0, + max_parallel_replicas = 3, + distributed_group_by_no_merge = 0, + enable_parallel_replicas = 1, + parallel_replicas_custom_key = 'k', + parallel_replicas_mode = 'custom_key_range', + parallel_replicas_custom_key_range_lower = 500, + parallel_replicas_custom_key_range_upper = 10000 ); -SELECT count() -FROM +SELECT count() +FROM ( SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) - SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, - parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', - parallel_replicas_custom_key_range_lower = 10, parallel_replicas_custom_key_range_upper = 13 + SETTINGS prefer_localhost_replica = 0, + max_parallel_replicas = 3, + distributed_group_by_no_merge = 0, + enable_parallel_replicas = 1, + parallel_replicas_custom_key = 'k', + parallel_replicas_mode = 'custom_key_range', + parallel_replicas_custom_key_range_lower = 10, + parallel_replicas_custom_key_range_upper = 13 ); DROP TABLE range_filter_custom_range_test; @@ -94,14 +134,19 @@ CREATE TABLE range_filter_custom_range_test_2 (k UInt64) ENGINE=MergeTree ORDER INSERT INTO range_filter_custom_range_test_2 SELECT number from numbers(13); -SELECT count() -FROM +SELECT count() +FROM ( SELECT * FROM cluster(parallel_replicas, currentDatabase(), range_filter_custom_range_test_2) - SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 12, distributed_group_by_no_merge = 0, - parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', - parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 13 + SETTINGS prefer_localhost_replica = 0, + max_parallel_replicas = 12, + distributed_group_by_no_merge = 0, + enable_parallel_replicas = 1, + parallel_replicas_custom_key = 'k', + parallel_replicas_mode = 'custom_key_range', + parallel_replicas_custom_key_range_lower = 0, + parallel_replicas_custom_key_range_upper = 13 ); DROP TABLE range_filter_custom_range_test_2; @@ -112,14 +157,19 @@ CREATE TABLE range_filter_custom_range_test_3 (k UInt64) ENGINE=MergeTree ORDER INSERT INTO range_filter_custom_range_test_3 SELECT number from numbers(4); -SELECT count() -FROM +SELECT count() +FROM ( SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test_3) - SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 12, distributed_group_by_no_merge = 0, - parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', - parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 4 + SETTINGS prefer_localhost_replica = 0, + max_parallel_replicas = 12, + distributed_group_by_no_merge = 0, + enable_parallel_replicas = 1, + parallel_replicas_custom_key = 'k', + parallel_replicas_mode = 'custom_key_range', + parallel_replicas_custom_key_range_lower = 0, + parallel_replicas_custom_key_range_upper = 4 ); -DROP TABLE range_filter_custom_range_test_3; \ No newline at end of file +DROP TABLE range_filter_custom_range_test_3; diff --git a/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql index da4626ad8970..d86334a236b0 100644 --- a/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql +++ b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql @@ -6,6 +6,7 @@ CREATE TABLE 03147_db.t (n Int8) ENGINE=MergeTree ORDER BY n; INSERT INTO 03147_db.t SELECT * FROM numbers(10); USE 03147_db; +-- We use the old setting here just to make sure we preserve it as an alias. SET allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 100; SELECT * FROM loop(03147_db.t) LIMIT 15 FORMAT Null; diff --git a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh index af7025697949..289a49c72f49 100755 --- a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh +++ b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh @@ -16,8 +16,8 @@ $CLICKHOUSE_CLIENT -q " INSERT INTO data2 VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-02', 'CREATED'); " -$CLICKHOUSE_CLIENT -nq " -SET enable_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 10, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; +$CLICKHOUSE_CLIENT -q " +SET enable_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 10, enable_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; SELECT id, diff --git a/tests/queries/0_stateless/03215_parallel_replicas_crash_after_refactoring.reference b/tests/queries/0_stateless/03215_parallel_replicas_crash_after_refactoring.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03215_parallel_replicas_crash_after_refactoring.sql b/tests/queries/0_stateless/03215_parallel_replicas_crash_after_refactoring.sql new file mode 100644 index 000000000000..240203b2a7c6 --- /dev/null +++ b/tests/queries/0_stateless/03215_parallel_replicas_crash_after_refactoring.sql @@ -0,0 +1,34 @@ +-- Tags: no-s3-storage +-- no-s3-storage: the test checks that there is no crash when we set parallel_replicas_single_task_marks_count_multiplier to 0 and we have some custom heuristic for +-- finding its optimal value in case if the table is stored on remote disk. + +DROP TABLE IF EXISTS 03215_parallel_replicas; + +CREATE TABLE 03215_parallel_replicas +( + `k` Int16, + `v` Nullable(UInt8) +) +ENGINE = MergeTree +ORDER BY k +SETTINGS index_granularity = 10; + +INSERT INTO 03215_parallel_replicas SELECT + number, + number +FROM numbers(1000); + +INSERT INTO 03215_parallel_replicas SELECT + number, + number +FROM numbers(1000, 1000); + +INSERT INTO 03215_parallel_replicas SELECT + number, + number +FROM numbers(2000, 1000); + +SET parallel_distributed_insert_select = 2, prefer_localhost_replica = false, enable_parallel_replicas = 1, max_parallel_replicas = 65535, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_single_task_marks_count_multiplier = -0., parallel_replicas_for_non_replicated_merge_tree = true; +SELECT max(k) IGNORE NULLS FROM 03215_parallel_replicas WITH TOTALS SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 65535, prefer_localhost_replica = 0, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_single_task_marks_count_multiplier = -0; -- { serverError 36 } + +DROP TABLE IF EXISTS 03215_parallel_replicas; diff --git a/tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.reference b/tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.sql b/tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.sql new file mode 100644 index 000000000000..6c2e05c12a90 --- /dev/null +++ b/tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS test_00808; + +CREATE TABLE test_00808 +( + `date` Date, + `id` Int8, + `name` String, + `value` Int64, + `sign` Int8 +) +ENGINE = CollapsingMergeTree(sign) +ORDER BY (id, date); + +INSERT INTO test_00808 VALUES('2000-01-01', 1, 'test string 1', 1, 1); +INSERT INTO test_00808 VALUES('2000-01-01', 2, 'test string 2', 2, 1); + +SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; + +SELECT * FROM (SELECT * FROM test_00808 FINAL) WHERE id = 1; -- { serverError SUPPORT_IS_DISABLED } + +DROP TABLE test_00808; diff --git a/tests/queries/0_stateless/03222_pr_asan_index_granularity.reference b/tests/queries/0_stateless/03222_pr_asan_index_granularity.reference new file mode 100644 index 000000000000..662880055bdb --- /dev/null +++ b/tests/queries/0_stateless/03222_pr_asan_index_granularity.reference @@ -0,0 +1,100 @@ +0 18 9899 +0 18 9898 +0 18 9897 +0 18 9896 +0 18 9895 +0 18 9894 +0 18 9893 +0 18 9892 +0 18 9891 +0 18 9890 +0 18 9889 +0 18 9888 +0 18 9887 +0 18 9886 +0 18 9885 +0 18 9884 +0 18 9883 +0 18 9882 +0 18 9881 +0 18 9880 +0 18 9879 +0 18 9878 +0 18 9877 +0 18 9876 +0 18 9875 +0 18 9874 +0 18 9873 +0 18 9872 +0 18 9871 +0 18 9870 +0 18 9869 +0 18 9868 +0 18 9867 +0 18 9866 +0 18 9865 +0 18 9864 +0 18 9863 +0 18 9862 +0 18 9861 +0 18 9860 +0 18 9859 +0 18 9858 +0 18 9857 +0 18 9856 +0 18 9855 +0 18 9854 +0 18 9853 +0 18 9852 +0 18 9851 +0 18 9850 +0 18 9849 +0 18 9848 +0 18 9847 +0 18 9846 +0 18 9845 +0 18 9844 +0 18 9843 +0 18 9842 +0 18 9841 +0 18 9840 +0 18 9839 +0 18 9838 +0 18 9837 +0 18 9836 +0 18 9835 +0 18 9834 +0 18 9833 +0 18 9832 +0 18 9831 +0 18 9830 +0 18 9829 +0 18 9828 +0 18 9827 +0 18 9826 +0 18 9825 +0 18 9824 +0 18 9823 +0 18 9822 +0 18 9821 +0 18 9820 +0 18 9819 +0 18 9818 +0 18 9817 +0 18 9816 +0 18 9815 +0 18 9814 +0 18 9813 +0 18 9812 +0 18 9811 +0 18 9810 +0 18 9809 +0 18 9808 +0 18 9807 +0 18 9806 +0 18 9805 +0 18 9804 +0 18 9803 +0 18 9802 +0 18 9801 +0 18 9800 diff --git a/tests/queries/0_stateless/03222_pr_asan_index_granularity.sql b/tests/queries/0_stateless/03222_pr_asan_index_granularity.sql new file mode 100644 index 000000000000..b7f37dd28568 --- /dev/null +++ b/tests/queries/0_stateless/03222_pr_asan_index_granularity.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test (k UInt64, v String) +ENGINE = MergeTree +ORDER BY k +SETTINGS index_granularity=1; + +INSERT INTO test SELECT number, toString(number) FROM numbers(10_000); + +SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; + +SELECT 0, materialize(18), k FROM test PREWHERE toNullable(toNullable(11)) WHERE toNullable(11) ORDER BY k DESC NULLS LAST LIMIT 100, 100 SETTINGS optimize_read_in_order = 1, merge_tree_min_rows_for_concurrent_read = 9223372036854775806, max_threads = 1; + +-- DROP TABLE test; diff --git a/tests/queries/0_stateless/03228_pr_subquery_view_order_by.reference b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.reference new file mode 100644 index 000000000000..aad720e27cfa --- /dev/null +++ b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.reference @@ -0,0 +1,20 @@ +300 +299 +298 +297 +296 +295 +294 +293 +292 +291 +290 +289 +288 +287 +286 +285 +284 +283 +282 +281 diff --git a/tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql new file mode 100644 index 000000000000..804a97f737f2 --- /dev/null +++ b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS view1; +DROP TABLE IF EXISTS table1; +CREATE TABLE table1 (number UInt64) ENGINE=MergeTree ORDER BY number SETTINGS index_granularity=1; +INSERT INTO table1 SELECT number FROM numbers(1, 300); +CREATE VIEW view1 AS SELECT number FROM table1; + +SELECT * +FROM +( + SELECT * + FROM view1 +) +ORDER BY number DESC +LIMIT 20 +SETTINGS cluster_for_parallel_replicas = 'parallel_replicas', allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree = 1, parallel_replicas_local_plan = 1; + +DROP TABLE view1; +DROP TABLE table1; diff --git a/tests/queries/1_stateful/00177_memory_bound_merging.reference b/tests/queries/1_stateful/00177_memory_bound_merging.reference index 5689152d60a1..4c7505d11232 100644 --- a/tests/queries/1_stateful/00177_memory_bound_merging.reference +++ b/tests/queries/1_stateful/00177_memory_bound_merging.reference @@ -10,6 +10,8 @@ http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-18 http:/ http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-19 http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-20 http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 1 -SortingAggregatedTransform MergingAggregatedBucketTransform -GroupingAggregatedTransform +FinishAggregatingInOrderTransform +FinalizeAggregatedTransform +AggregatingInOrderTransform +MergeTreeSelect diff --git a/tests/queries/1_stateful/00177_memory_bound_merging.sh b/tests/queries/1_stateful/00177_memory_bound_merging.sh index d5cd1a05cd8d..d256f814f15e 100755 --- a/tests/queries/1_stateful/00177_memory_bound_merging.sh +++ b/tests/queries/1_stateful/00177_memory_bound_merging.sh @@ -31,7 +31,7 @@ test1() { GROUP BY CounterID, URL, EventDate ORDER BY URL, EventDate LIMIT 5 OFFSET 10 - SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3" + SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, enable_parallel_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3" check_replicas_read_in_order $query_id } @@ -48,14 +48,14 @@ test2() { GROUP BY URL, EventDate ORDER BY URL, EventDate LIMIT 5 OFFSET 10 - SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, query_plan_aggregation_in_order = 1" + SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, enable_parallel_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, query_plan_aggregation_in_order = 1" check_replicas_read_in_order $query_id } test3() { $CLICKHOUSE_CLIENT -nq " SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; - SET max_threads = 16, prefer_localhost_replica = 1, read_in_order_two_level_merge_threshold = 1000, query_plan_aggregation_in_order = 1, distributed_aggregation_memory_efficient = 1; + SET max_threads = 16, read_in_order_two_level_merge_threshold = 1000, query_plan_aggregation_in_order = 1, distributed_aggregation_memory_efficient = 1; SELECT replaceRegexpOne(explain, '^ *(\w+).*', '\\1') FROM ( @@ -64,7 +64,7 @@ test3() { FROM test.hits WHERE CounterID = 1704509 AND UserID = 4322253409885123546 GROUP BY URL, EventDate - SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3 + SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, enable_parallel_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, parallel_replicas_local_plan=1 ) WHERE explain LIKE '%Aggr%Transform%' OR explain LIKE '%InOrder%'" }