From 62527482a71d213e1d5b1779b81524c4e2a7c64b Mon Sep 17 00:00:00 2001 From: madianjun Date: Mon, 6 Dec 2021 17:09:42 +0800 Subject: [PATCH 01/84] Add distributed plan, distributed source operator, grpc communication. Authors: madianjun Chao Ma caspian Allen Zhang --- programs/client/Client.cpp | 2 +- programs/server/Server.cpp | 21 + src/Client/GRPCClient.cpp | 121 +++++ src/Client/GRPCClient.h | 64 +++ src/Client/Suggest.cpp | 2 +- src/Common/ErrorCodes.cpp | 1 + src/Core/Defines.h | 10 + src/Databases/DatabaseReplicatedWorker.cpp | 2 +- src/Databases/DatabaseReplicatedWorker.h | 2 +- src/Interpreters/ClientInfo.h | 1 + src/Interpreters/Cluster.cpp | 132 ++++++ src/Interpreters/Cluster.h | 37 ++ src/Interpreters/Context.cpp | 29 ++ src/Interpreters/Context.h | 30 ++ src/Interpreters/DDLTask.cpp | 8 + src/Interpreters/DDLTask.h | 3 + src/Interpreters/DDLWorker.cpp | 34 +- src/Interpreters/DDLWorker.h | 3 +- src/Interpreters/DatabaseCatalog.cpp | 7 + src/Interpreters/DatabaseCatalog.h | 1 + src/Interpreters/InterpreterAlterQuery.cpp | 53 ++- src/Interpreters/InterpreterAlterQuery.h | 1 + src/Interpreters/InterpreterCreateQuery.cpp | 82 +++- src/Interpreters/InterpreterCreateQuery.h | 1 + src/Interpreters/InterpreterDropQuery.cpp | 30 ++ src/Interpreters/InterpreterSelectQuery.cpp | 5 +- .../InterpreterSelectWithUnionQuery.cpp | 5 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 49 +- src/Interpreters/executeDDLQueryOnCluster.h | 3 +- src/Interpreters/executeQuery.cpp | 11 +- src/Interpreters/executeQuery.h | 3 +- src/Interpreters/loadMetadata.cpp | 42 +- src/Interpreters/loadMetadata.h | 3 + src/Parsers/IAST.h | 3 + .../QueryPlan/DistributedSourceStep.cpp | 48 ++ .../QueryPlan/DistributedSourceStep.h | 38 ++ src/Processors/QueryPlan/QueryPlan.cpp | 444 +++++++++++++++++- src/Processors/QueryPlan/QueryPlan.h | 52 ++ src/Processors/Sources/DistributedSource.cpp | 89 ++++ src/Processors/Sources/DistributedSource.h | 37 ++ .../DistributedSourceExecutor.cpp | 90 ++++ src/QueryPipeline/DistributedSourceExecutor.h | 51 ++ src/Server/GRPCServer.cpp | 315 ++++++++++++- src/Server/grpc_protos/clickhouse_grpc.proto | 21 + 44 files changed, 1949 insertions(+), 37 deletions(-) create mode 100644 src/Client/GRPCClient.cpp create mode 100644 src/Client/GRPCClient.h create mode 100644 src/Processors/QueryPlan/DistributedSourceStep.cpp create mode 100644 src/Processors/QueryPlan/DistributedSourceStep.h create mode 100644 src/Processors/Sources/DistributedSource.cpp create mode 100644 src/Processors/Sources/DistributedSource.h create mode 100644 src/QueryPipeline/DistributedSourceExecutor.cpp create mode 100644 src/QueryPipeline/DistributedSourceExecutor.h diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 4951106f5957..2f0e873cf67e 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -329,7 +329,7 @@ bool Client::executeMultiQuery(const String & all_queries_text) std::vector Client::loadWarningMessages() { std::vector messages; - connection->sendQuery(connection_parameters.timeouts, "SELECT message FROM system.warnings", "" /* query_id */, + connection->sendQuery(connection_parameters.timeouts, "SELECT message FROM system.warnings", "zzzzzzzz-zzzz-zzzz-zzzz-zzzzzzzzzzzz" /* query_id */, QueryProcessingStage::Complete, nullptr, nullptr, false); while (true) { diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ce7d6973f68c..a7406d4c044e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -509,6 +509,22 @@ if (ThreadFuzzer::instance().isEffective()) global_context->addWarningMessage("Server was built with sanitizer. It will work slowly."); #endif + if (config().has("running_mode")) + { + String running_mode = config().getString("running_mode", "compute"); + if (running_mode == "compute") + { + global_context->setRunningMode(Context::RunningMode::COMPUTE); + LOG_INFO(log, "Running mode is COMPUTE"); + } + else if (running_mode == "store") + { + global_context->setRunningMode(Context::RunningMode::STORE); + LOG_INFO(log, "Running mode is STORE"); + } else { + throw Exception("Invalid running_mode", ErrorCodes::INVALID_CONFIG_PARAMETER); + } + } // Initialize global thread pool. Do it before we fetch configs from zookeeper // nodes (`from_zk`), because ZooKeeper interface uses the pool. We will @@ -1138,6 +1154,8 @@ if (ThreadFuzzer::instance().isEffective()) try { + /// New server download metadata from keeper, and add to server. + downloadMetaData(global_context); auto & database_catalog = DatabaseCatalog::instance(); /// We load temporary database first, because projections need it. database_catalog.initializeAndLoadTemporaryDatabase(); @@ -1550,6 +1568,9 @@ if (ThreadFuzzer::instance().isEffective()) server.start(); LOG_INFO(log, "Ready for connections."); + /// Register with clickhouse-keeper and watch on clusters change + global_context->setClustersWatcher(std::make_unique(DEFAULT_ZOOKEEPER_CLUSTERS_PATH, global_context, "ClustersWatcher")); + SCOPE_EXIT_SAFE({ LOG_DEBUG(log, "Received termination signal."); LOG_DEBUG(log, "Waiting for current connections to close."); diff --git a/src/Client/GRPCClient.cpp b/src/Client/GRPCClient.cpp new file mode 100644 index 000000000000..633a4f79fcfe --- /dev/null +++ b/src/Client/GRPCClient.cpp @@ -0,0 +1,121 @@ +#include +#include +#include +#include +#include +#include +#include "Common/ErrorCodes.h" +#include "base/logger_useful.h" +#include "clickhouse_grpc.grpc.pb.h" +#include "clickhouse_grpc.pb.h" + +namespace Poco +{ +class Logger; +} + +using GRPCQueryInfo = clickhouse::grpc::QueryInfo; +using GRPCResult = clickhouse::grpc::Result; +using GRPCException = clickhouse::grpc::Exception; +using GRPCTicket = clickhouse::grpc::Ticket; + +namespace DB +{ +namespace ErrorCodes +{ + extern const int INVALID_GRPC_QUERY_INFO; + extern const int GRPC_READ_ERROR; +} + +GRPCClient::GRPCClient(const String & _addr) +{ + addr = _addr; + + log = &Poco::Logger::get("GRPCClient"); +} + +GRPCClient::~GRPCClient() +{ + reader_map.clear(); +} + +GRPCResult GRPCClient::SendDistributedPlanParams(GRPCQueryInfo & gQueryInfo) +{ + auto ch = grpc::CreateChannel(addr, grpc::InsecureChannelCredentials()); + auto stub = clickhouse::grpc::ClickHouse::NewStub(ch); + grpc::ClientContext ctx; + GRPCResult result; + /// Set to native format, cause we decode result by NativeReader in the read function + gQueryInfo.set_output_format("Native"); + grpc::Status status = stub->SendDistributedPlanParams(&ctx, gQueryInfo, &result); + + if (status.ok()) + return result; + else + { + LOG_ERROR( + log, "SendDistributedPlanParams to {} failed with code {}, query_id: {}", addr, status.error_code(), gQueryInfo.query_id()); + throw Exception(status.error_message() + ", " + result.exception().display_text(), ErrorCodes::INVALID_GRPC_QUERY_INFO, true); + } +} + +Block GRPCClient::read(const GRPCTicket & ticket) +{ + String key = ticket.query_id() + "/" + std::to_string(ticket.stage_id()) + "/" + ticket.node_id(); + std::shared_ptr inner_ctx; + + std::shared_lock lock(mu); + auto reader_it = reader_map.find(key); + if (reader_it != reader_map.end()) + inner_ctx = reader_it->second; + lock.unlock(); + + if (reader_it == reader_map.end()) { + std::unique_lock wlock(mu); + /// Check again + reader_it = reader_map.find(key); + if (reader_it != reader_map.end()) + inner_ctx = reader_it->second; + else + { + auto ch = grpc::CreateChannel(addr, grpc::InsecureChannelCredentials()); + std::shared_ptr ctx = std::make_shared(); + auto stub = clickhouse::grpc::ClickHouse::NewStub(ch); + auto cReader = stub->ExecuteQueryFragmentWithStreamOutput(ctx.get(), ticket); + reader_map[key] = std::make_shared(ch, ctx, stub, cReader); + reader_it = reader_map.find(key); + inner_ctx = reader_it->second; + } + wlock.unlock(); + } + + LOG_DEBUG(log, "Read begin from {}", addr); + + GRPCResult result; + if (reader_it->second->reader->Read(&result)) + { + LOG_DEBUG(log, "Read result from {} success, exception.code: {}", addr, result.exception().code()); + if (result.exception().code() != 0) + { + LOG_ERROR(log, "GRPC addr: {} result exception: {} {}", addr, result.exception().code(), result.exception().display_text()); + throw Exception(result.exception().display_text(), ErrorCodes::INVALID_GRPC_QUERY_INFO, true); + } + + LOG_DEBUG(log, "Read from {} success, output size {}", addr, result.output().size()); + + if (result.output().size() == 0) + return {}; /// Read EOF + + ReadBufferFromString b(result.output()); + NativeReader reader(b, 0 /* server_revision_ */); + Block block = reader.read(); + LOG_DEBUG(log, "Read from {}, block decode success, has {} rows", addr, block.rows()); + return block; + } + + std::unique_lock wlock(mu); + reader_map.erase(reader_it); + wlock.unlock(); + throw Exception("read from grpc server failed! server: " + addr + ", " + std::to_string(result.exception().code()) + ", " + result.exception().display_text(), ErrorCodes::GRPC_READ_ERROR, true); +} +} diff --git a/src/Client/GRPCClient.h b/src/Client/GRPCClient.h new file mode 100644 index 000000000000..73a7a1b2a6c5 --- /dev/null +++ b/src/Client/GRPCClient.h @@ -0,0 +1,64 @@ +#pragma once +//#if USE_GRPC +#include +#include +#include + +#include +#include +#include + +#include +#include +#include +#include "clickhouse_grpc.grpc.pb.h" +#include "clickhouse_grpc.pb.h" + +using GRPCQueryInfo = clickhouse::grpc::QueryInfo; +using GRPCResult = clickhouse::grpc::Result; +using GRPCTicket = clickhouse::grpc::Ticket; +using GRPCStub = clickhouse::grpc::ClickHouse::Stub; + +namespace DB +{ +using ReadDataCallback = std::function; + +class GRPCClient +{ +public: + GRPCClient(const String & addr); + ~GRPCClient(); + + /// Send distributed plan to other servers. + GRPCResult SendDistributedPlanParams(GRPCQueryInfo & g_query_info); + + /// Try to read a block from the remote server by the specified ticket, + /// If got EOF, an empty Block will be returned, you can use if (!block) to check it. + Block read(const GRPCTicket & ticket); + +private: + struct InnerContext + { + InnerContext( + std::shared_ptr & ch_, + std::shared_ptr & ctx_, + std::unique_ptr & stub_, + std::unique_ptr> & reader_) + : ch(ch_), ctx(ctx_), stub(std::move(stub_)), reader(std::move(reader_)) + { + } + ~InnerContext() { } + + std::shared_ptr ch; + std::shared_ptr ctx; + std::unique_ptr stub; + std::unique_ptr> reader; + }; + + Poco::Logger * log; + String addr; + std::map> reader_map; + mutable std::shared_mutex mu; +}; +} +//#endif diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index b14af7ba8e90..170d66486efd 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -141,7 +141,7 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & timeouts, const std::string & query) { - connection.sendQuery(timeouts, query, "" /* query_id */, QueryProcessingStage::Complete, nullptr, nullptr, false); + connection.sendQuery(timeouts, query, "zzzzzzzz-zzzz-zzzz-zzzz-zzzzzzzzzzzz" /* query_id */, QueryProcessingStage::Complete, nullptr, nullptr, false); while (true) { diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 3cd914a89bb6..b1002e49cf92 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -593,6 +593,7 @@ M(623, CAPN_PROTO_BAD_CAST) \ M(624, BAD_FILE_TYPE) \ M(625, IO_SETUP_ERROR) \ + M(702, GRPC_READ_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 215bf6780d95..459dc0132a34 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -2,6 +2,14 @@ #include +#define DEFAULT_ZOOKEEPER_METADATA_PATH "/clickhouse/metadata" +#define CLUSTER_TYPE_COMPUTE "compute" +#define CLUSTER_TYPE_STORE "store" +/// Placeholder, meaningless name +#define CLUSTER_TYPE_ALL "all" +#define DEFAULT_ZOOKEEPER_CLUSTERS_PATH "/clickhouse/clusters" +#define STORE_NAMESPACE "shared-store" + #define DBMS_DEFAULT_PORT 9000 #define DBMS_DEFAULT_SECURE_PORT 9440 #define DBMS_DEFAULT_CONNECT_TIMEOUT_SEC 10 @@ -65,6 +73,8 @@ /// Max depth of hierarchical dictionary #define DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH 1000 +#define DEFAULT_ZOOKEEPER_CLUSTERS_PATH "/clickhouse/clusters" + /// Query profiler cannot work with sanitizers. /// Sanitizers are using quick "frame walking" stack unwinding (this implies -fno-omit-frame-pointer) /// And they do unwinding frequently (on every malloc/free, thread/mutex operations, etc). diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 365a5d028165..b5e847f7ff75 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -71,7 +71,7 @@ void DatabaseReplicatedDDLWorker::initializeReplication() last_skipped_entry_name.emplace(DDLTaskBase::getLogEntryName(our_log_ptr)); } -String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry) +String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry, const String & , const String & ) { auto zookeeper = getAndSetZooKeeper(); return enqueueQueryImpl(zookeeper, entry, database); diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 773612e403c0..bb83fab77b17 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -23,7 +23,7 @@ class DatabaseReplicatedDDLWorker : public DDLWorker public: DatabaseReplicatedDDLWorker(DatabaseReplicated * db, ContextPtr context_); - String enqueueQuery(DDLLogEntry & entry) override; + String enqueueQuery(DDLLogEntry & entry, const String & meta_path = "", const String & meta_info = "") override; String tryEnqueueAndExecuteEntry(DDLLogEntry & entry, ContextPtr query_context); diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index d42c34f07e2e..ad93e09ca183 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -57,6 +57,7 @@ class ClientInfo /// When query_kind == INITIAL_QUERY, these values are equal to current. String initial_user; + String initial_query; String initial_query_id; Poco::Net::SocketAddress initial_address; time_t initial_query_start_time{}; diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 30d0dd4cecee..dff4a76bb174 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -1,10 +1,13 @@ #include +#include #include #include #include #include #include #include +#include +#include #include #include #include @@ -720,4 +723,133 @@ bool Cluster::maybeCrossReplication() const return false; } +ClustersWatcher::ClustersWatcher(const std::string & clusters_path_, ContextPtr context_, const String & logger_name_) + : context(Context::createCopy(context_)), log(&Poco::Logger::get(logger_name_)) +{ + clusters_path = clusters_path_; + if (clusters_path.back() == '/') + clusters_path.resize(clusters_path.size() - 1); + + task = context->getSchedulePool().createTask("ClustersWatcher", [this] { run(); }); +} + +void ClustersWatcher::startup() +{ + /// Register with clickhouse-keeper + zkutil::ZooKeeperPtr zookeeper = context->getZooKeeper(); + while (true) + { + try + { + /// Create the clusters path if not exist. + zookeeper->createAncestors(clusters_path + "/"); + break; + } + catch (...) + { + LOG_ERROR(log, "ZooKeeper error: {}. Failed to create {}.", clusters_path, getCurrentExceptionMessage(true)); + } + sleepForSeconds(5); + } + + String node_path; + String replica = context->getMacros()->getValue("replica"); + if (context->getRunningMode() == Context::RunningMode::COMPUTE) + { + String my_namespace = context->getMacros()->getValue("namespace"); + node_path = clusters_path + "/compute_" + my_namespace + "_" + replica; + } + else + { + String shard = context->getMacros()->getValue("shard"); + node_path = clusters_path + "/store_" + shard + "_" + replica; + } + + UInt16 grpc_port = context->getServerPort("grpc_port"); + String node_data = replica + ":" + toString(grpc_port); + + Coordination::Requests requests; + requests.emplace_back(zkutil::makeCreateRequest(node_path, node_data, zkutil::CreateMode::Ephemeral)); + + /// If this fails, then throw on error and die. + zookeeper->multi(requests); + LOG_INFO(log, "Registered with clickhouse-keeper at {}.", node_path); + + task->activateAndSchedule(); +} + +void ClustersWatcher::run() +{ + auto parseReplica = [] (const String & key, ReplicaInfo & replica) { + int i = key.find("_"); + replica.type = key.substr(0, i); + int j = key.find("_", i + 1); + replica.group = key.substr(i + 1, j - (i + 1)); + replica.name = key.substr(j + 1, key.length() - (j + 1)); + }; + try + { + zkutil::ZooKeeper::Ptr zookeeper = context->getZooKeeper(); + Strings replica_znodes + = zookeeper->getChildrenWatch(clusters_path, nullptr, task->getWatchCallback()); + std::set replica_znodes_set(replica_znodes.begin(), replica_znodes.end()); + + /// Update clusters cache + { + std::lock_guard lock(mutex); + + /// Remove outdated replicas + std::erase_if(impl, [&replica_znodes_set, this](const auto & replica) { + const auto & [key, value] = replica; + bool should_removed = !replica_znodes_set.contains(key); + if (should_removed) { + LOG_DEBUG(log, "Remove replica {} from cluster cache.", key); + } + return should_removed; + }); + + /// Add new replicas + zkutil::AsyncResponses futures; + for (const auto & znode : replica_znodes) + { + + if (!impl.contains(znode)) + { + futures.emplace_back(znode, zookeeper->asyncGet(fs::path(clusters_path) / znode)); + } + } + for (auto & future : futures) + { + Coordination::GetResponse res = future.second.get(); + auto replica = std::make_shared(); + parseReplica(future.first, *replica); + replica->address = res.data; + impl[future.first] = replica; + LOG_DEBUG(log, "Add replica {} to cluster cache.", future.first); + } + LOG_DEBUG(log, "Cluster cache size: {}.", impl.size()); + } + } + catch (const Coordination::Exception & e) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + + if (e.code == Coordination::Error::ZSESSIONEXPIRED) + return; + + task->scheduleAfter(1000); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + task->scheduleAfter(1000); + } +} + +ClustersWatcher::Impl ClustersWatcher::getContainer() const +{ + std::lock_guard lock(mutex); + return impl; +} + } diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index ec78abf574c7..d6167242884b 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -2,6 +2,8 @@ #include #include +#include +#include #include @@ -303,4 +305,39 @@ class Clusters mutable std::mutex mutex; }; +class ClustersWatcher +{ +public: + ClustersWatcher(const std::string & clusters_path_, ContextPtr context_, + const String & logger_name_ = "ClustersWatcher"); + + void startup(); + void shutdown() { task->deactivate(); } + + void run(); + + struct ReplicaInfo { + String type; /// compute or store + String group; /// for compute replica, it's namespace; for store replica, it's shard + String name; /// replica name + bool operator==(const ReplicaInfo & other) const + { + return type == other.type && group == other.group && name == other.name; + } + String address; + }; + using ReplicaInfoPtr = std::shared_ptr; + using Impl = std::unordered_map; + + Impl getContainer() const; + +protected: + String clusters_path; + ContextMutablePtr context; + Poco::Logger * log; + BackgroundSchedulePool::TaskHolder task; + Impl impl; + mutable std::mutex mutex; +}; + } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7f31df9159ca..756b0c74fe2e 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -228,6 +228,7 @@ struct ContextSharedPart MultiVersion macros; /// Substitutions extracted from config. std::unique_ptr ddl_worker; /// Process ddl commands from zk. + std::unique_ptr clusters_watcher; /// Rules for selecting the compression settings, depending on the size of the part. mutable std::unique_ptr compression_codec_selector; /// Storage disk chooser for MergeTree engines @@ -273,6 +274,8 @@ struct ContextSharedPart Context::ApplicationType application_type = Context::ApplicationType::SERVER; + Context::RunningMode running_mode = Context::RunningMode::COMPUTE; + /// vector of xdbc-bridge commands, they will be killed when Context will be destroyed std::vector> bridge_commands; @@ -371,6 +374,7 @@ struct ContextSharedPart distributed_schedule_pool.reset(); message_broker_schedule_pool.reset(); ddl_worker.reset(); + clusters_watcher.reset(); access_control_manager.reset(); /// Stop trace collector if any @@ -1795,6 +1799,20 @@ DDLWorker & Context::getDDLWorker() const return *shared->ddl_worker; } +void Context::setClustersWatcher(std::unique_ptr clusters_watcher) +{ + auto lock = getLock(); + if (shared->clusters_watcher) + throw Exception("ClustersWatcher has already been initialized", ErrorCodes::LOGICAL_ERROR); + clusters_watcher->startup(); + shared->clusters_watcher = std::move(clusters_watcher); +} + +ClustersWatcher & Context::getClustersWatcher() const +{ + return *shared->clusters_watcher; +} + zkutil::ZooKeeperPtr Context::getZooKeeper() const { std::lock_guard lock(shared->zookeeper_mutex); @@ -2638,6 +2656,17 @@ void Context::setApplicationType(ApplicationType type) shared->application_type = type; } +Context::RunningMode Context::getRunningMode() const +{ + return shared->running_mode; +} + +void Context::setRunningMode(RunningMode mode) +{ + /// Lock isn't required, you should set it at start + shared->running_mode = mode; +} + void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & config) { shared->default_profile_name = config.getString("default_profile", "default"); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index cc6df875f7d3..4a5427a6136f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -54,6 +55,7 @@ class BackgroundSchedulePool; class MergeList; class ReplicatedFetchList; class Cluster; +class ClustersWatcher; class Compiler; class MarkCache; class MMappedFileCache; @@ -294,10 +296,22 @@ class Context: public std::enable_shared_from_this public: + struct QueryPlanFragmentInfo + { + String query_id; + int stage_id; /// Stage that should be executed on this replica. + int parent_stage_id; + String node_id; /// This is myself replica name and grpc port. + std::vector> sources; /// Point to replicas that sending data. + std::vector> sinks; /// Point to replicas that receiving data. + }; + // Top-level OpenTelemetry trace context for the query. Makes sense only for a query context. OpenTelemetryTraceContext query_trace_context; private: + std::optional query_plan_fragment_info; /// It has no value if current node is initial compute node. + using SampleBlockCache = std::unordered_map; mutable SampleBlockCache sample_block_cache; @@ -500,6 +514,10 @@ class Context: public std::enable_shared_from_this const QueryFactoriesInfo & getQueryFactoriesInfo() const { return query_factories_info; } void addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const; + bool isInitialComputeNode() const { return !query_plan_fragment_info; } + const QueryPlanFragmentInfo & getQueryPlanFragmentInfo() const { return query_plan_fragment_info.value(); } + void setQueryPlanFragmentInfo(const QueryPlanFragmentInfo & query_plan_fragment_info_) { query_plan_fragment_info = query_plan_fragment_info_; } + StoragePtr executeTableFunction(const ASTPtr & table_expression); void addViewSource(const StoragePtr & storage); @@ -741,6 +759,9 @@ class Context: public std::enable_shared_from_this void setDDLWorker(std::unique_ptr ddl_worker); DDLWorker & getDDLWorker() const; + void setClustersWatcher(std::unique_ptr clusters_watcher); + ClustersWatcher & getClustersWatcher() const; + std::shared_ptr getClusters() const; std::shared_ptr getCluster(const std::string & cluster_name) const; std::shared_ptr tryGetCluster(const std::string & cluster_name) const; @@ -827,6 +848,15 @@ class Context: public std::enable_shared_from_this ApplicationType getApplicationType() const; void setApplicationType(ApplicationType type); + enum class RunningMode + { + COMPUTE, /// compute server (default) + STORE, /// store server + }; + + RunningMode getRunningMode() const; + void setRunningMode(RunningMode mode); + /// Sets default_profile and system_profile, must be called once during the initialization void setDefaultProfiles(const Poco::Util::AbstractConfiguration & config); String getDefaultProfileName() const; diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index cfce6ac94639..6594650e73de 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -197,6 +197,14 @@ void DDLTask::setClusterInfo(ContextPtr context, Poco::Logger * log) throw Exception("Received unknown DDL query", ErrorCodes::UNKNOWN_TYPE_OF_QUERY); cluster_name = query_on_cluster->cluster; + /// If all cluster, ignore check + if (cluster_name == CLUSTER_TYPE_ALL) + { + query = query_on_cluster->getRewrittenASTWithoutOnCluster(address_in_cluster.default_database); + query_on_cluster = nullptr; + return; + } + cluster = context->tryGetCluster(cluster_name); if (!cluster) diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index ee49274707a8..8dedb5d97597 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -40,6 +40,9 @@ struct HostID HostID() = default; + explicit HostID(const String & host_name_, const UInt16 port_) + : host_name(host_name_), port(port_) {} + explicit HostID(const Cluster::Address & address) : host_name(address.host_name), port(address.port) {} diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index d7984af30c72..0391c94cba76 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -519,7 +519,7 @@ bool DDLWorker::tryExecuteQuery(const String & query, DDLTaskBase & task, const auto query_context = task.makeQueryContext(context, zookeeper); if (!task.is_initial_query) query_scope.emplace(query_context); - executeQuery(istr, ostr, !task.is_initial_query, query_context, {}); + executeQuery(istr, ostr, !task.is_initial_query, query_context, {}, std::nullopt, false); if (auto txn = query_context->getZooKeeperMetadataTransaction()) { @@ -1042,7 +1042,7 @@ void DDLWorker::createStatusDirs(const std::string & node_path, const ZooKeeperP } -String DDLWorker::enqueueQuery(DDLLogEntry & entry) +String DDLWorker::enqueueQuery(DDLLogEntry & entry, const String & meta_path, const String & meta_info) { if (entry.hosts.empty()) throw Exception("Empty host list in a distributed DDL task", ErrorCodes::LOGICAL_ERROR); @@ -1051,8 +1051,34 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry) String query_path_prefix = fs::path(queue_dir) / "query-"; zookeeper->createAncestors(query_path_prefix); + String node_path; + if (meta_path.empty()) + { + node_path = zookeeper->create(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential); + } + else + { + /// Atomic operation to create dll and save meta info + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential)); + if (meta_info.empty()) + { + /// Drop + ops.emplace_back(zkutil::makeRemoveRequest(meta_path, -1)); + } + else + { + if(zookeeper->exists(meta_path)) /// Alter + ops.emplace_back(zkutil::makeSetRequest(meta_path, meta_info, -1)); + else /// Create + ops.emplace_back(zkutil::makeCreateRequest(meta_path, meta_info, zkutil::CreateMode::Persistent)); + } + + /// If this fails, then we'll just retry from the start. + auto responses = zookeeper->multi(ops); + node_path = dynamic_cast(*responses[0]).path_created; + } - String node_path = zookeeper->create(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential); if (max_pushed_entry_metric) { String str_buf = node_path.substr(query_path_prefix.length()); @@ -1074,11 +1100,9 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry) { LOG_INFO(log, "An error occurred while creating auxiliary ZooKeeper directories in {} . They will be created later. Error : {}", node_path, getCurrentExceptionMessage(true)); } - return node_path; } - bool DDLWorker::initializeMainThread() { assert(!initialized); diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index d2b7c9d169de..c3fa8652c88e 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -48,7 +48,8 @@ class DDLWorker virtual ~DDLWorker(); /// Pushes query into DDL queue, returns path to created node - virtual String enqueueQuery(DDLLogEntry & entry); + /// Synchronize with the meta service if meta_path not empty + virtual String enqueueQuery(DDLLogEntry & entry, const String & meta_path = "", const String & meta_info = ""); /// Host ID (name:port) for logging purposes /// Note that in each task hosts are identified individually by name:port from initiator server cluster config diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 205d7052d0c7..c1b15564b225 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include "config_core.h" @@ -451,6 +452,12 @@ bool DatabaseCatalog::isTableExist(const DB::StorageID & table_id, ContextPtr co return db && db->isTableExist(table_id.table_name, context_); } +bool DatabaseCatalog::isTableExist(const String & table_path, ContextPtr context_) const +{ + auto zookeeper = context_->getZooKeeper(); + return zookeeper->exists(table_path); +} + void DatabaseCatalog::assertTableDoesntExist(const StorageID & table_id, ContextPtr context_) const { if (isTableExist(table_id, context_)) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 6079553b0255..69454bc6ab71 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -164,6 +164,7 @@ class DatabaseCatalog : boost::noncopyable, WithMutableContext /// For all of the following methods database_name in table_id must be not empty (even for temporary tables). void assertTableDoesntExist(const StorageID & table_id, ContextPtr context) const; bool isTableExist(const StorageID & table_id, ContextPtr context) const; + bool isTableExist(const String & table_path, ContextPtr context) const; bool isDictionaryExist(const StorageID & table_id) const; StoragePtr getTable(const StorageID & table_id, ContextPtr context) const; diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 729a495987fe..2e4abf1a8393 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -15,9 +15,11 @@ #include #include #include +#include #include #include #include +#include #include @@ -33,10 +35,12 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; extern const int NOT_IMPLEMENTED; extern const int TABLE_IS_READ_ONLY; + extern const int CANNOT_ASSIGN_ALTER; } -InterpreterAlterQuery::InterpreterAlterQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) +InterpreterAlterQuery::InterpreterAlterQuery(const ASTPtr & query_ptr_, ContextPtr context_) + : WithContext(context_), query_ptr(query_ptr_), log(&Poco::Logger::get("InterpreterAlterQuery")) { } @@ -158,6 +162,53 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) } } + if (alter.is_initial && !alter.table.empty()) + { + const_cast(alter).database = alter.database.empty() ? getContext()->getCurrentDatabase() : alter.database; + auto zookeeper = getContext()->getZooKeeper(); + String path = fs::path(DEFAULT_ZOOKEEPER_METADATA_PATH) / alter.database / alter.table; + if (!DatabaseCatalog::instance().isTableExist(path, getContext())) + { + throw Exception("Table " + alter.database + "." + alter.table + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + } + if(!alter_commands.empty()) + { + String alter_lock_path = fs::path(path) / "lock"; + Coordination::Error code = zookeeper->tryCreate(alter_lock_path, "", zkutil::CreateMode::Ephemeral); + if (code == Coordination::Error::ZNODEEXISTS) + { + throw Exception("Table " + alter.database + "." + alter.table + " is altered by another one, couldn't acquire lock", ErrorCodes::CANNOT_ASSIGN_ALTER); + } + auto alter_lock = zkutil::EphemeralNodeHolder::existing(alter_lock_path, *zookeeper); + + /// Format alter commands to sql + auto formatAlterCommandsToSql = [&alter, &table] (const String & statement, ContextPtr context, AlterCommands & alter_commands) + { + const_cast(alter).cluster.clear(); + StorageInMemoryMetadata metadata = table->getInMemoryMetadata(); + alter_commands.validate(metadata, context); + alter_commands.prepare(metadata); + table->checkAlterIsPossible(alter_commands, context); + StorageInMemoryMetadata new_metadata = table->getInMemoryMetadata(); + alter_commands.apply(new_metadata, context); + ParserCreateQuery parser; + ASTPtr ast = parseQuery( + parser, + statement.data(), + statement.data() + statement.size(), + "in file ", + 0, + context->getSettingsRef().max_parser_depth); + applyMetadataChangesToCreateQuery(ast, new_metadata); + return getObjectDefinitionFromCreateQuery(ast); + }; + String meta_info = formatAlterCommandsToSql(zookeeper->get(path), getContext(), alter_commands); + const_cast(alter).cluster = CLUSTER_TYPE_ALL; + executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess(), path, meta_info); + } + return {}; + } + if (!alter_commands.empty()) { StorageInMemoryMetadata metadata = table->getInMemoryMetadata(); diff --git a/src/Interpreters/InterpreterAlterQuery.h b/src/Interpreters/InterpreterAlterQuery.h index 9494a400e7b3..da57f12be032 100644 --- a/src/Interpreters/InterpreterAlterQuery.h +++ b/src/Interpreters/InterpreterAlterQuery.h @@ -34,6 +34,7 @@ class InterpreterAlterQuery : public IInterpreter, WithContext BlockIO executeToDatabase(const ASTAlterQuery & alter); ASTPtr query_ptr; + Poco::Logger * log; }; } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6d38c55bd62c..14b2feea40c9 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include @@ -93,7 +94,7 @@ namespace ErrorCodes namespace fs = std::filesystem; InterpreterCreateQuery::InterpreterCreateQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) - : WithMutableContext(context_), query_ptr(query_ptr_) + : WithMutableContext(context_), query_ptr(query_ptr_), log(&Poco::Logger::get("InterpreterCreateQuery")) { } @@ -753,7 +754,13 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const ErrorCodes::INCORRECT_QUERY); if (as_create.storage) + { create.set(create.storage, as_create.storage->ptr()); + if (create.storage->engine->name == "ReplicatedMergeTree") + { + create.storage->engine->arguments.reset(); + } + } else if (as_create.as_table_function) create.as_table_function = as_create.as_table_function->clone(); else @@ -1291,6 +1298,79 @@ BlockIO InterpreterCreateQuery::execute() { FunctionNameNormalizer().visit(query_ptr.get()); auto & create = query_ptr->as(); + String current_database = getContext()->getCurrentDatabase(); + create.database = create.database.empty() ? current_database : create.database; + if (create.is_initial && !this->internal) + { + auto zookeeper = getContext()->getZooKeeper(); + auto meta_path = DEFAULT_ZOOKEEPER_METADATA_PATH; + zookeeper->createIfNotExists(meta_path, String("")); + String path = fs::path(meta_path) / create.database; + if (!create.table.empty()) + { + path = fs::path(path) / create.table; + } + LOG_DEBUG(log, "Meta path: {}", path); + if (DatabaseCatalog::instance().isTableExist(path, getContext())) + { + if (create.if_not_exists) + return {}; + else + { + if (create.table.empty()) + throw Exception("Database " + create.database + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS); + else + throw Exception("Table " + create.database + "." + create.table + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); + } + + } + create.cluster = CLUSTER_TYPE_STORE; + prepareOnClusterQuery(create, getContext(), create.cluster); + ASTCreateQuery create_tmp = {create}; + String meta_info; + { + /// Format ast to sql + create.attach = true; + create.if_not_exists = false; + create.as_database.clear(); + create.as_table.clear(); + create.is_populate = false; + create.replace_view = false; + create.replace_table = false; + create.create_or_replace = false; + create.format = nullptr; + create.out_file = nullptr; + create.cluster.clear(); + if (create.table.empty()) + { + /// Database + create.database = TABLE_WITH_UUID_NAME_PLACEHOLDER; + if (!create.storage) + { + bool old_style_database = getContext()->getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary; + auto engine = std::make_shared(); + auto storage = std::make_shared(); + engine->name = old_style_database ? "Ordinary" : "Atomic"; + engine->no_empty_args = true; + storage->set(storage->engine, engine); + create.set(create.storage, storage); + } + } + else + { + /// Table + create.database.clear(); + create.table = TABLE_WITH_UUID_NAME_PLACEHOLDER; + } + meta_info = queryToString(create); + } + create = create_tmp; + /// All cluster to execute + create.cluster = CLUSTER_TYPE_ALL; + LOG_DEBUG(log, "DDL query on cluster: {}, create {}.{}", create.cluster, create.database, create.table); + executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess(), path, meta_info); + return {}; + } if (!create.cluster.empty()) { prepareOnClusterQuery(create, getContext(), create.cluster); diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 89d27a30555d..14ee3cad3a22 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -93,6 +93,7 @@ class InterpreterCreateQuery : public IInterpreter, WithMutableContext void assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const; ASTPtr query_ptr; + Poco::Logger * log; /// Skip safety threshold when loading tables. bool has_force_restore_data_flag = false; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 4fbad7e5471c..cf7cf4b5cb7a 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -11,6 +11,8 @@ #include #include #include +#include +#include #include "config_core.h" @@ -22,6 +24,7 @@ # include #endif +namespace fs = std::filesystem; namespace DB { @@ -30,6 +33,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int SYNTAX_ERROR; extern const int UNKNOWN_TABLE; + extern const int UNKNOWN_DATABASE; extern const int NOT_IMPLEMENTED; extern const int INCORRECT_QUERY; extern const int TABLE_IS_READ_ONLY; @@ -50,6 +54,32 @@ InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, ContextMut BlockIO InterpreterDropQuery::execute() { auto & drop = query_ptr->as(); + if (drop.table.empty() && drop.database.empty()) + throw Exception("Nothing to drop, both names are empty", ErrorCodes::LOGICAL_ERROR); + if (drop.is_initial) + { + auto zookeeper = getContext()->getZooKeeper(); + drop.database = drop.database.empty() ? getContext()->getCurrentDatabase() : drop.database; + String path = fs::path(DEFAULT_ZOOKEEPER_METADATA_PATH) / drop.database; + if (!drop.table.empty()) + path = fs::path(path) / drop.table; + + if (!DatabaseCatalog::instance().isTableExist(path, getContext())) + { + if (!drop.if_exists) + { + if (drop.table.empty()) + throw Exception("Database " + drop.database + " doesn't exist.", ErrorCodes::UNKNOWN_DATABASE); + else + throw Exception("Table " + drop.database + "." + drop.table + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + } + else + return {}; + } + drop.cluster = CLUSTER_TYPE_ALL; + executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccessForDDLOnCluster(), path, ""); + return {}; + } if (!drop.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccessForDDLOnCluster()); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0050df1bf52d..22f61a71b4c4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -594,8 +594,11 @@ BlockIO InterpreterSelectQuery::execute() buildQueryPlan(query_plan); + query_plan.buildDistributedPlan(context); + + QueryPlanOptimizationSettings do_not_optimize_plan{.optimize_plan = false}; res.pipeline = QueryPipelineBuilder::getPipeline(std::move(*query_plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)))); + do_not_optimize_plan, BuildQueryPipelineSettings::fromContext(context)))); return res; } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index e7ea08e557d3..88eb38a402d0 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -328,8 +328,11 @@ BlockIO InterpreterSelectWithUnionQuery::execute() QueryPlan query_plan; buildQueryPlan(query_plan); + query_plan.buildDistributedPlan(context); + + QueryPlanOptimizationSettings do_not_optimize_plan{.optimize_plan = false}; auto pipeline_builder = query_plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), + do_not_optimize_plan, BuildQueryPipelineSettings::fromContext(context)); pipeline_builder->addInterpreterContext(context); diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index c3b8cc5c6776..7136b793542b 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -59,7 +59,8 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, c return executeDDLQueryOnCluster(query_ptr, context, AccessRightsElements{query_requires_access}); } -BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, AccessRightsElements && query_requires_access) +BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, AccessRightsElements && query_requires_access, + const String& meta_path, const String& meta_info) { /// Remove FORMAT and INTO OUTFILE if exists ASTPtr query_ptr = query_ptr_->clone(); @@ -84,17 +85,43 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, } } - query->cluster = context->getMacros()->expand(query->cluster); - ClusterPtr cluster = context->getCluster(query->cluster); + bool is_all_cluster = query->cluster == CLUSTER_TYPE_ALL; DDLWorker & ddl_worker = context->getDDLWorker(); - - /// Enumerate hosts which will be used to send query. - Cluster::AddressesWithFailover shards = cluster->getShardsAddresses(); std::vector hosts; - for (const auto & shard : shards) + Cluster::AddressesWithFailover shards; + if (!is_all_cluster) + { + query->cluster = context->getMacros()->expand(query->cluster); + ClusterPtr cluster = context->getCluster(query->cluster); + /// Enumerate hosts which will be used to send query. + shards = cluster->getShardsAddresses(); + for (const auto & shard : shards) + { + for (const auto & addr : shard) + hosts.emplace_back(addr); + } + } + else { - for (const auto & addr : shard) - hosts.emplace_back(addr); + if (meta_path.empty()) + throw Exception("Empty meta path in a distributed DDL task", ErrorCodes::LOGICAL_ERROR); + /// Get Hosts from meta service + auto getHostsFromMetaService = [&context](std::vector& hosts){ + auto zookeeper = context->getZooKeeper(); + auto clusters_path = DEFAULT_ZOOKEEPER_CLUSTERS_PATH; + Strings children = zookeeper->getChildren(clusters_path); + for(auto& child: children) + { + auto host_port_str = zookeeper->get(fs::path(clusters_path) / child); + auto pos = host_port_str.find(':'); + if (pos == std::string::npos) + continue; + auto host = host_port_str.substr(0, pos); + HostID hostId(host, context->getTCPPort()); + hosts.emplace_back(std::move(hostId)); + } + }; + getHostsFromMetaService(hosts); } if (hosts.empty()) @@ -110,7 +137,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, bool use_local_default_database = false; const String & current_database = context->getCurrentDatabase(); - if (need_replace_current_database) + if (!is_all_cluster && need_replace_current_database) { Strings shard_default_databases; for (const auto & shard : shards) @@ -163,7 +190,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, entry.query = queryToString(query_ptr); entry.initiator = ddl_worker.getCommonHostID(); entry.setSettingsIfRequired(context); - String node_path = ddl_worker.enqueueQuery(entry); + String node_path = ddl_worker.enqueueQuery(entry, meta_path, meta_info); return getDistributedDDLStatus(node_path, entry, context); } diff --git a/src/Interpreters/executeDDLQueryOnCluster.h b/src/Interpreters/executeDDLQueryOnCluster.h index e7ec52d03cbf..822521692b14 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.h +++ b/src/Interpreters/executeDDLQueryOnCluster.h @@ -25,7 +25,8 @@ bool isSupportedAlterType(int type); /// Returns DDLQueryStatusSource, which reads results of query execution on each host in the cluster. BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context); BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, const AccessRightsElements & query_requires_access); -BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, AccessRightsElements && query_requires_access); +BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, AccessRightsElements && query_requires_access, + const String& meta_path = "", const String& meta_info = ""); BlockIO getDistributedDDLStatus( const String & node_path, const DDLLogEntry & entry, ContextPtr context, const std::optional & hosts_to_wait = {}); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index f401f708ab14..d12431afc0a7 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -376,11 +376,13 @@ static std::tuple executeQueryImpl( ContextMutablePtr context, bool internal, QueryProcessingStage::Enum stage, - ReadBuffer * istr) + ReadBuffer * istr, + bool is_initial = true) { const auto current_time = std::chrono::system_clock::now(); auto & client_info = context->getClientInfo(); + client_info.initial_query = String(begin, end); // If it's not an internal query and we don't see an initial_query_start_time yet, initialize it // to current time. Internal queries are those executed without an independent client context, @@ -596,6 +598,8 @@ static std::tuple executeQueryImpl( return std::make_tuple(ast, std::move(io)); } + // is_initial flag + ast->is_initial = is_initial; auto interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); @@ -977,7 +981,8 @@ void executeQuery( bool allow_into_outfile, ContextMutablePtr context, SetResultDetailsFunc set_result_details, - const std::optional & output_format_settings) + const std::optional & output_format_settings, + bool is_initial) { PODArray parse_buf; const char * begin; @@ -1011,7 +1016,7 @@ void executeQuery( ASTPtr ast; BlockIO streams; - std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr); + std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr, is_initial); auto & pipeline = streams.pipeline; std::unique_ptr compressed_buffer; diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index 9c561d8b88cb..f96cc1fdcf58 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -20,7 +20,8 @@ void executeQuery( bool allow_into_outfile, /// If true and the query contains INTO OUTFILE section, redirect output to that file. ContextMutablePtr context, /// DB, tables, data types, storage engines, functions, aggregate functions... SetResultDetailsFunc set_result_details, /// If a non-empty callback is passed, it will be called with the query id, the content-type, the format, and the timezone. - const std::optional & output_format_settings = std::nullopt /// Format settings for output format, will be calculated from the context if not set. + const std::optional & output_format_settings = std::nullopt, /// Format settings for output format, will be calculated from the context if not set. + bool is_initial = true /// is initial flag ); diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 65b2065b2adf..06cd6550b8d4 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -9,7 +9,8 @@ #include #include #include - +#include +#include #include #include @@ -166,7 +167,9 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam TablesLoader::Databases loaded_databases; for (const auto & [name, db_path] : databases) { - loadDatabase(context, name, db_path, has_force_restore_data_flag); + auto db = DatabaseCatalog::instance().tryGetDatabase(name); + if (!db) + loadDatabase(context, name, db_path, has_force_restore_data_flag); loaded_databases.insert({name, DatabaseCatalog::instance().getDatabase(name)}); } @@ -231,4 +234,39 @@ void loadMetadataSystem(ContextMutablePtr context) /// Will startup tables in system database after all databases are loaded. } +void downloadMetaData(ContextMutablePtr context) +{ + auto zookeeper = context->getZooKeeper(); + auto meta_server_path = DEFAULT_ZOOKEEPER_METADATA_PATH; + zookeeper->createAncestors(fs::path(meta_server_path) / "default" / ""); + auto meta_local_path = fs::path(context->getPath()) / "metadata"; + auto database_names = zookeeper->getChildren(meta_server_path); + auto write_to_file = [](const String& file, const String& content){ + WriteBufferFromFile out(file, content.size(), O_WRONLY | O_CREAT | O_TRUNC); + writeString(content, out); + out.next(); + out.close(); + }; + for(auto& database : database_names) + { + String local_db_path = fs::path(meta_local_path)/database; + String meta_db_path = fs::path(meta_server_path)/database; + String attach_db = zookeeper->get(meta_db_path); + if (attach_db.empty()) + continue; + write_to_file(local_db_path + ".sql", attach_db); + loadDatabase(context, database, local_db_path, true); + auto table_names = zookeeper->getChildren(meta_db_path); + for (auto& table : table_names) + { + String local_tb_path = fs::path(local_db_path)/table; + local_tb_path += ".sql"; + auto attach_table = zookeeper->get(fs::path(meta_db_path)/table); + if (attach_table.empty()) + continue; + write_to_file(local_tb_path, attach_table); + } + } +} + } diff --git a/src/Interpreters/loadMetadata.h b/src/Interpreters/loadMetadata.h index e918b5f530c7..c13f22ac9ab7 100644 --- a/src/Interpreters/loadMetadata.h +++ b/src/Interpreters/loadMetadata.h @@ -19,4 +19,7 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam /// so we startup system tables after all databases are loaded. void startupSystemTables(); +/// Synchronize databases and tables from meta service +void downloadMetaData(ContextMutablePtr context); + } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 1c9c81ad3780..e2e2622fecf9 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -35,6 +35,9 @@ class IAST : public std::enable_shared_from_this, public TypePromotion +#include +#include + +namespace DB +{ + +DistributedSourceStep::DistributedSourceStep( + Block header_, + const std::vector> & sources_, + const String & query_id_, + int stage_id_, + int parent_stage_id_, + const String & node_id_, + ContextPtr context_) + : ISourceStep(DataStream{.header = std::move(header_)}) + , header(output_stream->header) + , sources(sources_) + , query_id(query_id_) + , stage_id(stage_id_) + , parent_stage_id(parent_stage_id_) + , node_id(node_id_) + , context(std::move(context_)) + , log(&Poco::Logger::get("DistributedSourceStep(" + query_id + "/" + toString(stage_id) + "/" + node_id + ")")) +{ +} + + +void DistributedSourceStep::addPipe(Pipes & pipes, const std::shared_ptr & source) +{ + auto distributed_source_executor = std::make_shared(header, source, query_id, node_id, parent_stage_id); + pipes.emplace_back(createDistributedSourcePipe(distributed_source_executor, false)); + pipes.back().addInterpreterContext(context); +} + +void DistributedSourceStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + Pipes pipes; + for (const auto & source : sources) + { + addPipe(pipes, source); + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + pipeline.init(std::move(pipe)); +} + +} diff --git a/src/Processors/QueryPlan/DistributedSourceStep.h b/src/Processors/QueryPlan/DistributedSourceStep.h new file mode 100644 index 000000000000..8b5bc547eed3 --- /dev/null +++ b/src/Processors/QueryPlan/DistributedSourceStep.h @@ -0,0 +1,38 @@ +#pragma once +#include +#include +#include + +namespace DB +{ +class DistributedSourceStep final : public ISourceStep +{ +public: + DistributedSourceStep( + Block header_, + const std::vector> & sources_, + const String & query_id_, + int stage_id_, + int parent_stage_id_, + const String & node_id_, + ContextPtr context_); + + String getName() const override { return "DistributedSourceStep"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + const std::vector> & getSources() const { return sources; } + + void setSources(const std::vector> & sources_) { sources = sources_; } + +private: + void addPipe(Pipes & pipes, const std::shared_ptr & source); + Block header; + std::vector> sources; + String query_id; + int stage_id; + int parent_stage_id; + String node_id; + ContextPtr context; + Poco::Logger * log; +}; +} + diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index f319e562bfbd..c285d813d0e3 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -11,6 +11,9 @@ #include #include #include +#include +#include +#include namespace DB { @@ -20,7 +23,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -QueryPlan::QueryPlan() = default; +QueryPlan::QueryPlan() : log(&Poco::Logger::get("QueryPlan")) +{ +} QueryPlan::~QueryPlan() = default; QueryPlan::QueryPlan(QueryPlan &&) = default; QueryPlan & QueryPlan::operator=(QueryPlan &&) = default; @@ -134,6 +139,443 @@ void QueryPlan::addStep(QueryPlanStepPtr step) " input expected", ErrorCodes::LOGICAL_ERROR); } +void QueryPlan::reset() +{ + root = nullptr; + nodes.clear(); +} + +void QueryPlan::buildStages(ContextPtr) +{ + LOG_DEBUG(log, "Build stages."); + + auto createStage = [this](int id, Stage * parent_stage, Node * node) { + stages.emplace_back(Stage{.id = id, .node = node}); + Stage * new_stage = &stages.back(); + if (parent_stage) + { + new_stage->parents.push_back(parent_stage); + parent_stage->child = new_stage; + } + return new_stage; + }; + + struct Frame + { + Node * node = {}; + int visited_children = 0; /// Number of visited children + }; + + /// Used for visiting the query plan tree. + std::stack stack; + stack.push(Frame{.node = root}); + bool one_child_is_visited = false; + + /// Used for creating stage. + int stage_id = -1; + Node * stage_root_node = nullptr; + Stage * last_stage = nullptr; + + while (!stack.empty()) + { + auto & frame = stack.top(); + + if (one_child_is_visited) + { + ++frame.visited_children; + one_child_is_visited = false; + + /// TODO: This is shuffle, construct a new stage + // if (false) + // { + // stage_id++; /// current stage + // last_stage = createStage(stage_id, last_stage); + // } + } + + size_t next_child = frame.visited_children; + if (next_child == frame.node->children.size()) + { + LOG_DEBUG(log, "Visited step: {}", frame.node->step->getName()); + stage_root_node = frame.node; + one_child_is_visited = true; + stack.pop(); + } + else + stack.push(Frame{.node = frame.node->children[next_child]}); + } + + /// At last, append a shuffle for converging data. + ++stage_id; + last_stage = createStage(stage_id, last_stage, stage_root_node); + + /// Create result stage. + ++stage_id; + result_stage = createStage(stage_id, last_stage, nullptr); +} + +void QueryPlan::scheduleStages(ContextPtr context) +{ + /// Retrieve all replicas. + std::unordered_map replicas = context->getClustersWatcher().getContainer(); + LOG_DEBUG(log, "Schedule stages across {} replicas.", replicas.size()); + std::vector> store_replicas, compute_replicas; + for (const auto & replica : replicas) + { + const auto & replica_info = replica.second; + LOG_DEBUG( + log, + "Check replica: {} => ({}/{}/{}, {}).", + replica.first, + replica_info->type, + replica_info->group, + replica_info->name, + replica_info->address); + + if (replica_info->type == "store") + { + store_replicas.emplace_back(std::make_shared(replica_info->address)); + } + else + { + compute_replicas.emplace_back(std::make_shared(replica_info->address)); + } + } + LOG_DEBUG(log, "{} store, {} compute.", store_replicas.size(), compute_replicas.size()); + + auto fillStage = [&store_replicas, &compute_replicas](Stage * stage) + { + if (stage->parents.empty()) /// Leaf stage. + { + /// Fill executors. + stage->executors.reserve(store_replicas.size()); + stage->executors.insert(stage->executors.end(), store_replicas.begin(), store_replicas.end()); + /// Leaf stage's sources should be empty. + } + else /// Non-leaf stage. + { + /// Fill executors. + stage->executors.reserve(compute_replicas.size()); + stage->executors.insert(stage->executors.end(), compute_replicas.begin(), compute_replicas.end()); + /// Fill sources. + int num_sources = 0; + for (Stage * parent : stage->parents) + { + num_sources += parent->executors.size(); + } + stage->sources.reserve(num_sources); + for (Stage * parent : stage->parents) + { + stage->sources.insert(stage->sources.end(), parent->executors.begin(), parent->executors.end()); + } + } + }; + + struct Frame + { + Stage * stage = {}; + int visited_parents = 0; /// Number of visited parents + }; + + assert(result_stage != nullptr); + std::stack stack; + stack.push(Frame{.stage = result_stage}); + + bool one_parent_is_visited = false; + while (!stack.empty()) + { + auto & frame = stack.top(); + + if (one_parent_is_visited) + { + ++frame.visited_parents; + one_parent_is_visited = false; + } + + size_t next_parent = frame.visited_parents; + if (next_parent == frame.stage->parents.size()) + { + LOG_DEBUG(log, "Visited stage: {}", frame.stage->id); + + fillStage(frame.stage); + + one_parent_is_visited = true; + stack.pop(); + } + else + { + stack.push(Frame{.stage = frame.stage->parents[next_parent]}); + } + } + + /// Send plan fragment params. + const String & query_id = context->getClientInfo().initial_query_id; + for (auto & stage : stages) + { + /// Don't send result stage. + if (&stage == result_stage) + { + assert(!result_stage->parents.empty()); + Block header; + const Stage * parent_stage = result_stage->parents[0]; + const Node * parent_stage_node = result_stage->parents[0]->node; + if (!parent_stage_node->step) + LOG_DEBUG(log, "Step of parent stage's node is empty."); + else if (!parent_stage_node->step->hasOutputStream()) + LOG_DEBUG(log, "Step of parent stage's node doesn't have output stream."); + else if (!parent_stage_node->step->getOutputStream().header) + LOG_DEBUG(log, "Step of parent stage's node has empty header."); + else + { + header = parent_stage_node->step->getOutputStream().header; + LOG_DEBUG( + log, + "Take the output stream header of {}: {}, header columns: {}.", + parent_stage_node->step->getName(), + parent_stage_node->step->getStepDescription(), + header.columns()); + } + + /// Get my replica grpc address + String my_replica = context->getMacros()->getValue("replica") + ":" + toString(context->getServerPort("grpc_port")); + auto distributed_source_step = std::make_unique( + header, parent_stage->executors, query_id, result_stage->id, parent_stage->id, my_replica, context); + { + /// Only for debug. + LOG_DEBUG( + log, + "Local plan fragment:\n{}", + debugLocalPlanFragment(query_id, result_stage->id, my_replica, distributed_source_step.get())); + } + reset(); + addStep(std::move(distributed_source_step)); + continue; + } + + /// Fill sinks. + if (!stage.child->executors.empty()) + { + stage.sinks.reserve(stage.child->executors.size()); + stage.sinks.insert(stage.sinks.end(), stage.child->executors.begin(), stage.child->executors.end()); + } + + LOG_DEBUG(log, "Stage {} has {} executors.", stage.id, stage.executors.size()); + /// Send to each remote executor. + for (const auto & executor : stage.executors) + { + LOG_DEBUG(log, "Plan fragment to send:\nquery: {}\n{}", context->getClientInfo().initial_query, debugRemotePlanFragment(*executor, query_id, &stage)); + +// const auto & query_distributed_plan_info = Context::QueryPlanFragmentInfo{ +// .query_id = query_id, .stage_id = stage->id, .node_id = *executor, .sources = stage->sources, .sinks = stage->sinks}; + GRPCQueryInfo query_info; + query_info.set_query(context->getClientInfo().initial_query); + query_info.set_query_id(query_id); + query_info.set_stage_id(stage.id); + if (!stage.parents.empty()) + query_info.set_parent_stage_id(stage.parents[0]->id); + query_info.set_node_id(*executor); +// query_info.set_user_name(context->getUserName()); +// query_info.set_password(context->getClientInfo().) + for (const auto & source : stage.sources) + { + query_info.add_sources(*source); + } + for (const auto & sink : stage.sinks) + { + query_info.add_sinks(*sink); + } + + GRPCClient cli(*executor); + auto result = cli.SendDistributedPlanParams(query_info); + LOG_INFO(log, "GRPCClient got result exception: {} {}.", result.exception().code(), result.exception().display_text()); + } + } +} + +void QueryPlan::buildPlanFragment(ContextPtr context) +{ + LOG_DEBUG(log, "Building plan fragment."); + + /// Get my replica grpc address + String my_replica = context->getMacros()->getValue("replica") + ":" + toString(context->getServerPort("grpc_port")); + const auto & query_distributed_plan_info = context->getQueryPlanFragmentInfo(); + int my_stage_id = query_distributed_plan_info.stage_id; +// int parent_stage_id = query_distributed_plan_info.parent_stage_id; + + struct Frame + { + Node * node = {}; + int visited_children = 0; /// Number of visited children + }; + + /// Used for visiting the query plan tree. + std::stack stack; + stack.push(Frame{.node = root}); + bool one_child_is_visited = false; + + /// Used for locating the plan fragment. + int stage_id = -1; + Node * fragment_root_node = nullptr; + Node * fragment_leaf_node = nullptr; /// TODO: For join or union, there are more than one leaf nodes. + + while (!stack.empty()) + { + auto & frame = stack.top(); + + if (one_child_is_visited) + { + ++frame.visited_children; + one_child_is_visited = false; + + /// TODO: This is a shuffle relationship between current node and the last visited child(i.e. fragment_root_node). +// if (true) +// { +// stage_id++; +// if (stage_id == my_stage_id) +// { +// root = fragment_root_node; +// DistributedSourceStep * step = nullptr; +// if (fragment_leaf_node) +// { +// /// Set sources for fragment_leaf_node +// step = dynamic_cast(fragment_leaf_node->step.get()); +// step->setSources(query_distributed_plan_info.sources); +// } +// +// { +// /// Only for debug. +// LOG_DEBUG( +// log, +// "Local plan fragment:\n{}", +// debugLocalPlanFragment(query_distributed_plan_info.query_id, stage_id, my_replica, step)); +// } +// return; +// } +// +// /// Add a DistributedSourceStep between current node and fragment_root_node. +// /// But this step may NOT be executed in my fragment. +// frame.node->children.clear(); +// const auto & header = fragment_root_node->step->getOutputStream().header; +// const String & query_id = context->getClientInfo().initial_query_id; +// auto distributed_source_step = std::make_unique(header, {}, query_id, my_stage_id, parent_stage_id, my_replica, context); +// insertStep(frame.node->step, distributed_source_step); +// nodes.emplace_back(Node{.step = std::move(step)}); +// fragment_leaf_node = &nodes.back(); +// frame.node->children.emplace_back(fragment_leaf_node); +// } + } + + size_t next_child = frame.visited_children; + if (next_child == frame.node->children.size()) + { + LOG_DEBUG(log, "Visited step: {}", frame.node->step->getName()); + + fragment_root_node = frame.node; + one_child_is_visited = true; + stack.pop(); + } + else + stack.push(Frame{.node = frame.node->children[next_child]}); + } + + /// Check the result stage. + ++stage_id; + if (stage_id == my_stage_id) + { + root = fragment_root_node; + DistributedSourceStep * step = nullptr; + if (fragment_leaf_node) + { + step = dynamic_cast(fragment_leaf_node->step.get()); + step->setSources(query_distributed_plan_info.sources); + } + { + /// Only for debug. + LOG_DEBUG( + log, + "Local plan fragment:\n{}", + debugLocalPlanFragment(query_distributed_plan_info.query_id, stage_id, my_replica, step)); + } + return; + } +} + +void QueryPlan::buildDistributedPlan(ContextMutablePtr context) +{ + if (context->getRunningMode() == Context::RunningMode::STORE) + return; + + if (context->getInitialQueryId() == "zzzzzzzz-zzzz-zzzz-zzzz-zzzzzzzzzzzz") + { + LOG_DEBUG(log, "Special initial query id, skip building distributed plan."); + return; + } + + LOG_DEBUG(log, "Initial query id: {}, to be built to distributed plan.", context->getInitialQueryId()); + +// /// For hard code debugging +// if (context->getMacros()->getValue("cluster") == "store") +// { +// auto query_plan_fragment_info +// = Context::QueryPlanFragmentInfo{.query_id = "xxx-yyy-zzz", .stage_id = 0, .node_id = "centos0"}; +// query_plan_fragment_info.sinks = {std::make_shared("ubuntu0")}; +// context->setQueryPlanFragmentInfo(query_plan_fragment_info); +// } + + checkInitialized(); + if (context->isInitialComputeNode()) + { + optimize(QueryPlanOptimizationSettings::fromContext(context)); + buildStages(context); + scheduleStages(context); + } + else + { + buildPlanFragment(context); + } +} + +String QueryPlan::debugLocalPlanFragment(const String & query_id, int stage_id, const String & node_id, const DistributedSourceStep * step) +{ + WriteBufferFromOwnString buf; + ExplainPlanOptions options; + buf << "fragment id: " << query_id << "/" << stage_id << "/" << node_id << "\n"; + if (step) + { + buf << "sources: "; + for (const auto & source : step->getSources()) + { + buf << *source << " "; + } + buf.write('\n'); + } + + explainPlan(buf, options); + return buf.str(); +} + +String QueryPlan::debugRemotePlanFragment(const String & receiver, const String & query_id, const Stage * stage) +{ + WriteBufferFromOwnString buf; + buf << "receiver: " << receiver; + buf.write('\n'); + buf << "fragment id: " << query_id << "/" << stage->id << "/" << receiver; + buf.write('\n'); + buf << "sources: "; + for (const auto & source : stage->sources) + { + buf << *source << " "; + } + buf.write('\n'); + buf << "sinks: "; + for (const auto & sink : stage->sinks) + { + buf << *sink << " "; + } + buf.write('\n'); + return buf.str(); +} + QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline( const QueryPlanOptimizationSettings & optimization_settings, const BuildQueryPipelineSettings & build_pipeline_settings) diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 4e342d746d1d..19088cd66b9b 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -56,6 +57,12 @@ class QueryPlan void optimize(const QueryPlanOptimizationSettings & optimization_settings); + void reset(); + void buildStages(ContextPtr context); /// Used by initial node. + void scheduleStages(ContextPtr context); /// Used by initial node. + void buildPlanFragment(ContextPtr context); /// Used by non-initial nodes. + void buildDistributedPlan(ContextMutablePtr context); + QueryPipelineBuilderPtr buildQueryPipeline( const QueryPlanOptimizationSettings & optimization_settings, const BuildQueryPipelineSettings & build_pipeline_settings); @@ -104,6 +111,34 @@ class QueryPlan using Nodes = std::list; + struct Stage + { + int id; + std::vector parents = {}; /// Previous stages that current stage directly depends on. + Stage * child = nullptr; + std::vector> executors; /// Replicas that current stage should be executed on. + std::vector> sources; /// Parents' executors. + std::vector> sinks; /// Child's executors. + Node * node; /// Current stage's root node + }; + + /// Note: do not use vector, otherwise pointers to elements in it will be invalidated when vector increases. + using Stages = std::list; + + struct PlanFragmentInfo + { + PlanFragmentInfo(int stage_id_, const String & node_id_, const std::vector & sources_, const std::vector & sinks_) + : stage_id(stage_id_), node_id(node_id_), sources(sources_), sinks(sinks_) {} + int stage_id; + String node_id; /// The replica name of plan fragment receiver, used by DistributedSource. + std::vector sources; /// Point to the nodes sending data. + std::vector sinks; /// Point to the nodes receiving data. + }; + using PlanFragmentInfoPtr = std::shared_ptr; + + String debugLocalPlanFragment(const String & query_id, int stage_id, const String & node_id, const DistributedSourceStep * stage); + String debugRemotePlanFragment(const String & receiver, const String & query_id, const Stage * stage); + private: Nodes nodes; Node * root = nullptr; @@ -114,6 +149,23 @@ class QueryPlan /// Those fields are passed to QueryPipeline. size_t max_threads = 0; std::vector interpreter_context; + + Stages stages; + Stage * result_stage = nullptr; + + struct pairHasher { + template + size_t operator()(const std::pair & p) const + { + auto hash1 = std::hash{}(p.first); + auto hash2 = std::hash{}(p.second); + return hash1 ^ hash2; + } + }; + + /// Key is {stage_id, receiver_address}. + std::unordered_map>, PlanFragmentInfoPtr, pairHasher> plan_fragment_infos; + Poco::Logger * log; }; std::string debugExplainStep(const IQueryPlanStep & step); diff --git a/src/Processors/Sources/DistributedSource.cpp b/src/Processors/Sources/DistributedSource.cpp new file mode 100644 index 000000000000..de0f1d37083f --- /dev/null +++ b/src/Processors/Sources/DistributedSource.cpp @@ -0,0 +1,89 @@ +#include +#include +#include + +namespace DB +{ +DistributedSource::DistributedSource(DistributedSourceExecutorPtr executor, bool async_read_) + : SourceWithProgress(executor->getHeader(), false) + , query_executor(std::move(executor)) + , async_read(async_read_) + , log(&Poco::Logger::get("DistributedSource")) +{ + LOG_DEBUG(log, "DistributedSource header columns: {}.", getPort().getHeader().columns()); +} + +DistributedSource::~DistributedSource() = default; + +ISource::Status DistributedSource::prepare() +{ + /// Check if query was cancelled before returning Async status. Otherwise it may lead to infinite loop. + if (was_query_canceled) + { + getPort().finish(); + return Status::Finished; + } + + if (is_async_state) + return Status::Async; + + Status status = SourceWithProgress::prepare(); + /// To avoid resetting the connection (because of "unfinished" query) in the + /// RemoteQueryExecutor it should be finished explicitly. + if (status == Status::Finished) + { + query_executor->finish(); + is_async_state = false; + } + return status; +} + +std::optional DistributedSource::tryGenerate() +{ + /// onCancel() will do the cancel if the query was sent. + if (was_query_canceled) + return {}; + + Block block; + + if (async_read) + { + /// do something if needed + } + else + block = query_executor->read(); + + if (!block) + { + query_executor->finish(); + return {}; + } + + UInt64 num_rows = block.rows(); + Chunk chunk(block.getColumns(), num_rows); + + return std::move(chunk); +} + +void DistributedSource::onCancel() +{ + was_query_canceled = true; + query_executor->cancel(); +} + +void DistributedSource::onUpdatePorts() +{ + if (getPort().isFinished()) + { + was_query_canceled = true; + query_executor->finish(); + } +} + +Pipe createDistributedSourcePipe(DistributedSourceExecutorPtr query_executor, bool async_read) +{ + Pipe pipe(std::make_shared(query_executor, async_read)); + return pipe; +} + +} diff --git a/src/Processors/Sources/DistributedSource.h b/src/Processors/Sources/DistributedSource.h new file mode 100644 index 000000000000..15f123d179cb --- /dev/null +++ b/src/Processors/Sources/DistributedSource.h @@ -0,0 +1,37 @@ +#pragma once +#include +#include + +namespace DB +{ +class DistributedSourceExecutor; +using DistributedSourceExecutorPtr = std::shared_ptr; + +class DistributedSource : public SourceWithProgress +{ +public: + DistributedSource(DistributedSourceExecutorPtr executor, + bool async_read_); + ~DistributedSource() override; + + Status prepare() override; + String getName() const override { return "Distributed"; } + + void onUpdatePorts() override; + +protected: + std::optional tryGenerate() override; + void onCancel() override; + +private: + DistributedSourceExecutorPtr query_executor; + std::atomic was_query_canceled = false; + const bool async_read; + Poco::Logger * log = nullptr; + bool is_async_state = false; +}; + +/// Create pipe with distributed sources. +Pipe createDistributedSourcePipe(DistributedSourceExecutorPtr query_executor, bool async_read); + +} diff --git a/src/QueryPipeline/DistributedSourceExecutor.cpp b/src/QueryPipeline/DistributedSourceExecutor.cpp new file mode 100644 index 000000000000..46488c070c2c --- /dev/null +++ b/src/QueryPipeline/DistributedSourceExecutor.cpp @@ -0,0 +1,90 @@ +#include +#include + +namespace DB +{ +DistributedSourceExecutor::DistributedSourceExecutor( + Block header_, + const std::shared_ptr & source_, + const String & query_id_, + const String & node_id_, + int stage_id_) + : header(std::move(header_)) + , source(source_) + , query_id(query_id_) + , node_id(node_id_) + , stage_id(stage_id_) + , client(*source_) + , log(&Poco::Logger::get("DistributedSourceExecutor(" + query_id + "/" + toString(stage_id) + "/" + node_id + ")")) +{ + ticket.set_query_id(query_id); + ticket.set_stage_id(stage_id); + ticket.set_node_id(node_id); +} + +DistributedSourceExecutor::~DistributedSourceExecutor() +{ +} + +Block DistributedSourceExecutor::read() +{ + if (was_cancelled) + return Block(); + + try + { + auto block = client.read(ticket); + LOG_DEBUG(log, "Read block, rows: {}, columns: {}.", block.rows(), block.columns()); + return block; + } + catch (...) + { + got_exception_from_replica = true; + throw; + } +} + +void DistributedSourceExecutor::finish() +{ + if (!isQueryPending() || hasThrownException()) + return; + + LOG_DEBUG(log, "Finish reading from {}.", *source); + tryCancel("Cancelling query because enough data has been read"); + + finished = true; +} + +void DistributedSourceExecutor::cancel() +{ + if (!isQueryPending() || hasThrownException()) + return; + + LOG_DEBUG(log, "Cancel reading from {}.", *source); + tryCancel("Cancelling query"); +} + +void DistributedSourceExecutor::tryCancel(const char * reason) +{ + /// Flag was_cancelled is atomic because it is checked in read(). + std::lock_guard guard(was_cancelled_mutex); + + if (was_cancelled) + return; + + was_cancelled = true; + + LOG_TRACE(log, "({}) {}", *source, reason); +} + +bool DistributedSourceExecutor::isQueryPending() const +{ + return !finished; +} + +bool DistributedSourceExecutor::hasThrownException() const +{ + return got_exception_from_replica; +} + +} diff --git a/src/QueryPipeline/DistributedSourceExecutor.h b/src/QueryPipeline/DistributedSourceExecutor.h new file mode 100644 index 000000000000..84c6adac1b1d --- /dev/null +++ b/src/QueryPipeline/DistributedSourceExecutor.h @@ -0,0 +1,51 @@ +#pragma once +#include +#include +#include + +namespace DB +{ +class DistributedSourceExecutor +{ +public: + /// Takes already set connection. + DistributedSourceExecutor( + Block header_, + const std::shared_ptr & source_, + const String & query_id_, + const String & node_id_, + int stage_id_); + + ~DistributedSourceExecutor(); + + Block read(); + void finish(); + void cancel(); + + const Block & getHeader() const { return header; } + +private: + Block header; + const std::shared_ptr source; + String query_id; + String node_id; + int stage_id; + GRPCTicket ticket; + GRPCClient client; + Poco::Logger * log = nullptr; + std::atomic finished{false}; + std::atomic was_cancelled{false}; + std::atomic got_exception_from_replica{false}; + std::mutex was_cancelled_mutex; + + /// If wasn't sent yet, send request to cancel all connections to replicas + void tryCancel(const char * reason); + + /// Returns true if query was sent + bool isQueryPending() const; + + /// Returns true if exception was thrown + bool hasThrownException() const; +}; + +} diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index ba2644e0fbae..ae659455ca54 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1,6 +1,7 @@ #include "GRPCServer.h" #include #include +#include #if USE_GRPC #include @@ -46,6 +47,7 @@ using GRPCService = clickhouse::grpc::ClickHouse::AsyncService; using GRPCQueryInfo = clickhouse::grpc::QueryInfo; +using GRPCTicket = clickhouse::grpc::Ticket; using GRPCResult = clickhouse::grpc::Result; using GRPCException = clickhouse::grpc::Exception; using GRPCProgress = clickhouse::grpc::Progress; @@ -272,6 +274,7 @@ namespace const CompletionCallback & callback) = 0; virtual void read(GRPCQueryInfo & query_info_, const CompletionCallback & callback) = 0; + virtual void read(GRPCTicket & ticket_, const CompletionCallback & callback) = 0; virtual void write(const GRPCResult & result, const CompletionCallback & callback) = 0; virtual void writeAndFinish(const GRPCResult & result, const grpc::Status & status, const CompletionCallback & callback) = 0; @@ -329,6 +332,8 @@ namespace CALL_WITH_STREAM_INPUT, /// ExecuteQueryWithStreamInput() call CALL_WITH_STREAM_OUTPUT, /// ExecuteQueryWithStreamOutput() call CALL_WITH_STREAM_IO, /// ExecuteQueryWithStreamIO() call + CALL_SEND_SIMPLE, /// SendDistributedPlanParams() call + CALL_FRAGMENT_WITH_STREAM_OUTPUT, /// ExecuteQueryFragmentWithStreamOutput() call CALL_MAX, }; @@ -340,6 +345,8 @@ namespace case CALL_WITH_STREAM_INPUT: return "ExecuteQueryWithStreamInput()"; case CALL_WITH_STREAM_OUTPUT: return "ExecuteQueryWithStreamOutput()"; case CALL_WITH_STREAM_IO: return "ExecuteQueryWithStreamIO()"; + case CALL_SEND_SIMPLE: return "SendDistributedPlanParams()"; + case CALL_FRAGMENT_WITH_STREAM_OUTPUT: return "ExecuteQueryFragmentWithStreamOutput()"; case CALL_MAX: break; } __builtin_unreachable(); @@ -352,7 +359,7 @@ namespace bool isOutputStreaming(CallType call_type) { - return (call_type == CALL_WITH_STREAM_OUTPUT) || (call_type == CALL_WITH_STREAM_IO); + return (call_type == CALL_WITH_STREAM_OUTPUT) || (call_type == CALL_WITH_STREAM_IO) || (call_type == CALL_FRAGMENT_WITH_STREAM_OUTPUT); } template @@ -379,6 +386,11 @@ namespace callback(true); } + void read(GRPCTicket &, const CompletionCallback &) override + { + throw Exception("Responder::read() should not be called", ErrorCodes::LOGICAL_ERROR); + } + void write(const GRPCResult &, const CompletionCallback &) override { throw Exception("Responder::write() should not be called", ErrorCodes::LOGICAL_ERROR); @@ -411,6 +423,11 @@ namespace reader.Read(&query_info_, getCallbackPtr(callback)); } + void read(GRPCTicket &, const CompletionCallback &) override + { + throw Exception("Responder::read() should not be called", ErrorCodes::LOGICAL_ERROR); + } + void write(const GRPCResult &, const CompletionCallback &) override { throw Exception("Responder::write() should not be called", ErrorCodes::LOGICAL_ERROR); @@ -446,6 +463,11 @@ namespace callback(true); } + void read(GRPCTicket &, const CompletionCallback &) override + { + throw Exception("Responder::read() should not be called", ErrorCodes::LOGICAL_ERROR); + } + void write(const GRPCResult & result, const CompletionCallback & callback) override { writer.Write(result, getCallbackPtr(callback)); @@ -478,6 +500,11 @@ namespace reader_writer.Read(&query_info_, getCallbackPtr(callback)); } + void read(GRPCTicket &, const CompletionCallback &) override + { + throw Exception("Responder::read() should not be called", ErrorCodes::LOGICAL_ERROR); + } + void write(const GRPCResult & result, const CompletionCallback & callback) override { reader_writer.Write(result, getCallbackPtr(callback)); @@ -492,6 +519,88 @@ namespace grpc::ServerAsyncReaderWriter reader_writer{&grpc_context}; }; + template<> + class Responder : public BaseResponder + { + public: + void start(GRPCService & grpc_service, + grpc::ServerCompletionQueue & new_call_queue, + grpc::ServerCompletionQueue & notification_queue, + const CompletionCallback & callback) override + { + grpc_service.RequestSendDistributedPlanParams(&grpc_context, &query_info.emplace(), &response_writer, &new_call_queue, ¬ification_queue, getCallbackPtr(callback)); + } + + void read(GRPCQueryInfo & query_info_, const CompletionCallback & callback) override + { + if (!query_info.has_value()) + callback(false); + query_info_ = std::move(query_info).value(); + query_info.reset(); + callback(true); + } + + void read(GRPCTicket &, const CompletionCallback &) override + { + throw Exception("Responder::read() should not be called", ErrorCodes::LOGICAL_ERROR); + } + + void write(const GRPCResult &, const CompletionCallback &) override + { + throw Exception("Responder::write() should not be called", ErrorCodes::LOGICAL_ERROR); + } + + void writeAndFinish(const GRPCResult & result, const grpc::Status & status, const CompletionCallback & callback) override + { + response_writer.Finish(result, status, getCallbackPtr(callback)); + } + + private: + grpc::ServerAsyncResponseWriter response_writer{&grpc_context}; + std::optional query_info; + }; + + template<> + class Responder : public BaseResponder + { + public: + void start(GRPCService & grpc_service, + grpc::ServerCompletionQueue & new_call_queue, + grpc::ServerCompletionQueue & notification_queue, + const CompletionCallback & callback) override + { + grpc_service.RequestExecuteQueryFragmentWithStreamOutput(&grpc_context, &ticket.emplace(), &writer, &new_call_queue, ¬ification_queue, getCallbackPtr(callback)); + } + + void read(GRPCQueryInfo &, const CompletionCallback &) override + { + throw Exception( "Responder::read() should not be called", ErrorCodes::LOGICAL_ERROR); + } + + void read(GRPCTicket & ticket_, const CompletionCallback & callback) override + { + if (!ticket.has_value()) + callback(false); + ticket_ = std::move(ticket).value(); + ticket.reset(); + callback(true); + } + + void write(const GRPCResult & result, const CompletionCallback & callback) override + { + writer.Write(result, getCallbackPtr(callback)); + } + + void writeAndFinish(const GRPCResult & result, const grpc::Status & status, const CompletionCallback & callback) override + { + writer.WriteAndFinish(result, {}, status, getCallbackPtr(callback)); + } + + private: + grpc::ServerAsyncWriter writer{&grpc_context}; + std::optional ticket; + }; + std::unique_ptr makeResponder(CallType call_type) { switch (call_type) @@ -500,6 +609,8 @@ namespace case CALL_WITH_STREAM_INPUT: return std::make_unique>(); case CALL_WITH_STREAM_OUTPUT: return std::make_unique>(); case CALL_WITH_STREAM_IO: return std::make_unique>(); + case CALL_SEND_SIMPLE: return std::make_unique>(); + case CALL_FRAGMENT_WITH_STREAM_OUTPUT: return std::make_unique>(); case CALL_MAX: break; } __builtin_unreachable(); @@ -562,6 +673,36 @@ namespace mutable std::condition_variable changed; }; + template + class InnerMap + { + public: + bool getAndErase(const String & key_, Value & value_) + { + std::lock_guard lock(mutex); + auto it = container.find(key_); + if (it == container.end()) + return false; + std::swap(it->second, value_); + container.erase(it); + return true; + } + + bool insert(const String& key_, Value && value_) + { + std::lock_guard lock(mutex); + if (container.find(key_) != container.end()) + return false; + container.emplace(key_, value_); + return true; + } + + private: + std::unordered_map container; + std::mutex mutex; + }; + + using QueryInfoMap = InnerMap; /// Handles a connection after a responder is started (i.e. after getting a new call). class Call @@ -575,7 +716,11 @@ namespace private: void run(); + void saveQueryInfo(); + void runPlanFragment(); + void receiveQuery(); + void receiveTicket(); void executeQuery(); void processInput(); @@ -591,7 +736,9 @@ namespace void close(); void readQueryInfo(); + void readTicket(); void throwIfFailedToReadQueryInfo(); + void throwIfFailedToReadTicket(); bool isQueryCancelled(); void addProgressToResult(); @@ -603,6 +750,8 @@ namespace void throwIfFailedToSendResult(); void sendException(const Exception & exception); + static std::unique_ptr query_info_map; + const CallType call_type; std::unique_ptr responder; IServer & iserver; @@ -627,8 +776,10 @@ namespace GRPCQueryInfo query_info; /// We reuse the same messages multiple times. GRPCResult result; + GRPCTicket ticket; bool initial_query_info_read = false; + bool initial_ticket_read = false; bool finalize = false; bool responder_finished = false; bool cancelled = false; @@ -648,8 +799,11 @@ namespace /// The following fields are accessed both from call_thread and queue_thread. BoolState reading_query_info{false}; + BoolState reading_ticket{false}; std::atomic failed_to_read_query_info = false; + std::atomic failed_to_read_ticket = false; GRPCQueryInfo next_query_info_while_reading; + GRPCTicket next_ticket_while_reading; std::atomic want_to_cancel = false; std::atomic check_query_info_contains_cancel_only = false; BoolState sending_result{false}; @@ -658,6 +812,8 @@ namespace ThreadFromGlobalPool call_thread; }; + std::unique_ptr Call::query_info_map = std::make_unique(); + Call::Call(CallType call_type_, std::unique_ptr responder_, IServer & iserver_, Poco::Logger * log_) : call_type(call_type_), responder(std::move(responder_)), iserver(iserver_), log(log_) { @@ -671,11 +827,19 @@ namespace void Call::start(const std::function & on_finish_call_callback) { - auto runner_function = [this, on_finish_call_callback] + void (Call::*function_to_run)() = nullptr; + if (call_type == CALL_SEND_SIMPLE) + function_to_run = &Call::saveQueryInfo; + else if (call_type == CALL_FRAGMENT_WITH_STREAM_OUTPUT) + function_to_run = &Call::runPlanFragment; + else + function_to_run = &Call::run; + + auto runner_function = [this, on_finish_call_callback, function_to_run] { try { - run(); + (this->*function_to_run)(); } catch (...) { @@ -711,6 +875,71 @@ namespace } } + void Call::saveQueryInfo() + { + try + { + setThreadName("GRPCServerCall"); + receiveQuery(); + auto query_id = query_info.query_id(); + auto stage_id = std::to_string(query_info.stage_id()); + auto sinks = query_info.sinks(); + /// TODO::Optimize flow + for(auto& node_id : sinks) + { + auto plan_fragment_id = query_id + "/" + stage_id + "/" + node_id; + LOG_DEBUG(log, "key: {}", plan_fragment_id); + auto status = query_info_map->insert(plan_fragment_id, std::move(query_info)); + if (!status) + { + throw Exception("Plan fragment id " + plan_fragment_id + " already exists", ErrorCodes::LOGICAL_ERROR); + } + } + finishQuery(); + } + catch (Exception & exception) + { + onException(exception); + } + catch (Poco::Exception & exception) + { + onException(Exception{Exception::CreateFromPocoTag{}, exception}); + } + catch (std::exception & exception) + { + onException(Exception{Exception::CreateFromSTDTag{}, exception}); + } + } + + void Call::runPlanFragment() + { + try + { + setThreadName("GRPCServerCall"); + receiveTicket(); + auto plan_fragment_id = ticket.query_id() + "/" + std::to_string(ticket.stage_id()) + "/" + ticket.node_id(); + auto status = query_info_map->getAndErase(plan_fragment_id, query_info); + if (!status) + throw Exception("Plan fragment id " + plan_fragment_id + " not exists", ErrorCodes::LOGICAL_ERROR); + executeQuery(); + processInput(); + generateOutput(); + finishQuery(); + } + catch (Exception & exception) + { + onException(exception); + } + catch (Poco::Exception & exception) + { + onException(Exception{Exception::CreateFromPocoTag{}, exception}); + } + catch (std::exception & exception) + { + onException(Exception{Exception::CreateFromSTDTag{}, exception}); + } + } + void Call::receiveQuery() { LOG_INFO(log, "Handling call {}", getCallName(call_type)); @@ -723,6 +952,15 @@ namespace LOG_DEBUG(log, "Received initial QueryInfo: {}", getQueryDescription(query_info)); } + void Call::receiveTicket() + { + LOG_INFO(log, "Handling call {}", getCallName(call_type)); + + readTicket(); + + LOG_DEBUG(log, "Received ticket(stream name): {}", ticket.query_id() + "/" + std::to_string(ticket.stage_id()) + "/" + ticket.node_id()); + } + void Call::executeQuery() { /// Retrieve user credentials. @@ -752,6 +990,29 @@ namespace query_context = session->makeQueryContext(); + auto genQueryPlanFragmentInfo = [](ContextMutablePtr & context_, GRPCQueryInfo & info_) + { + std::vector> sources, sinks; + for (auto & source : info_.sources()) + { + sources.emplace_back(std::make_shared(source)); + } + for (auto & sink : info_.sinks()) + { + sinks.emplace_back(std::make_shared(sink)); + } + Context::QueryPlanFragmentInfo fragmentInfo{ + .query_id = info_.query_id(), + .stage_id = info_.stage_id(), + .parent_stage_id = info_.parent_stage_id(), + .node_id = info_.node_id(), + .sources = sources, + .sinks = sinks }; + context_->setQueryPlanFragmentInfo(std::move(fragmentInfo)); + }; + + genQueryPlanFragmentInfo(query_context, query_info); + /// Prepare settings. SettingsChanges settings_changes; for (const auto & [key, value] : query_info.settings()) @@ -1177,7 +1438,10 @@ namespace finalize = true; io.onFinish(); addProgressToResult(); - query_scope->logPeakMemoryUsage(); + if (query_scope.has_value()) + { + query_scope->logPeakMemoryUsage(); + } addLogsToResult(); releaseQueryIDAndSessionID(); sendResult(); @@ -1329,6 +1593,41 @@ namespace } } + void Call::readTicket() + { + auto start_reading = [&] + { + reading_ticket.set(true); + responder->read(next_ticket_while_reading, [this](bool ok) + { + /// Called on queue_thread. + if (!ok) + { + failed_to_read_ticket = true; + } + reading_ticket.set(false); + }); + }; + auto finish_reading = [&] + { + if (reading_ticket.get()) + { + Stopwatch client_writing_watch; + reading_ticket.wait(false); + waited_for_client_writing += client_writing_watch.elapsedNanoseconds(); + } + throwIfFailedToReadTicket(); + ticket = std::move(next_ticket_while_reading); + initial_ticket_read = true; + }; + if (!initial_ticket_read) + { + start_reading(); + } + + finish_reading(); + } + void Call::throwIfFailedToReadQueryInfo() { if (failed_to_read_query_info) @@ -1340,6 +1639,14 @@ namespace } } + void Call::throwIfFailedToReadTicket() + { + if (failed_to_read_ticket) + { + throw Exception("Failed to read Ticket", ErrorCodes::NETWORK_ERROR); + } + } + bool Call::isQueryCancelled() { if (cancelled) diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index c6cafaf6e404..e76699608b9d 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -60,6 +60,14 @@ message Compression { CompressionLevel level = 2; } +message Ticket +{ + // same as QueryInfo.query_id + string query_id = 1; + int32 stage_id = 2; + string node_id = 3; +} + // Information about a query which a client sends to a ClickHouse server. // The first QueryInfo can set any of the following fields. Extra QueryInfos only add extra data. // In extra QueryInfos only `input_data`, `external_tables`, `next_query_info` and `cancel` fields can be set. @@ -101,6 +109,16 @@ message QueryInfo { /// Controls how a ClickHouse server will compress query execution results before sending back to the client. /// If not set the compression settings from the configuration file will be used. Compression result_compression = 17; + + /// the stage id of target host which receives this request + int32 stage_id = 18; + /// the id of the target host + string node_id = 19; + /// TODO use map to represent the relation between source and parent stage id + int32 parent_stage_id = 20; + repeated string sources = 21; + /// replica_name:grpc_port of sinks + repeated string sinks = 22; } enum LogsLevel { @@ -171,4 +189,7 @@ service ClickHouse { rpc ExecuteQueryWithStreamInput(stream QueryInfo) returns (Result) {} rpc ExecuteQueryWithStreamOutput(QueryInfo) returns (stream Result) {} rpc ExecuteQueryWithStreamIO(stream QueryInfo) returns (stream Result) {} + + rpc SendDistributedPlanParams(QueryInfo) returns (Result) {} + rpc ExecuteQueryFragmentWithStreamOutput(Ticket) returns (stream Result) {} } From 0704d1d8172c3f2286214cec95339545189b03d4 Mon Sep 17 00:00:00 2001 From: madianjun Date: Mon, 6 Dec 2021 15:46:36 +0800 Subject: [PATCH 02/84] Optimize building stages and plan fragment. --- .../QueryPlan/DistributedSourceStep.h | 2 +- src/Processors/QueryPlan/QueryPlan.cpp | 100 ++++++++---------- src/Processors/QueryPlan/QueryPlan.h | 2 +- 3 files changed, 44 insertions(+), 60 deletions(-) diff --git a/src/Processors/QueryPlan/DistributedSourceStep.h b/src/Processors/QueryPlan/DistributedSourceStep.h index 8b5bc547eed3..1dae70dbd933 100644 --- a/src/Processors/QueryPlan/DistributedSourceStep.h +++ b/src/Processors/QueryPlan/DistributedSourceStep.h @@ -17,7 +17,7 @@ class DistributedSourceStep final : public ISourceStep const String & node_id_, ContextPtr context_); - String getName() const override { return "DistributedSourceStep"; } + String getName() const override { return "DistributedSourceStep(" + toString(stage_id) + " -> " + toString(parent_stage_id); } void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; const std::vector> & getSources() const { return sources; } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index c285d813d0e3..5d07d6dfbb09 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -186,11 +186,11 @@ void QueryPlan::buildStages(ContextPtr) one_child_is_visited = false; /// TODO: This is shuffle, construct a new stage - // if (false) - // { - // stage_id++; /// current stage - // last_stage = createStage(stage_id, last_stage); - // } +// if (false) +// { +// stage_id++; +// last_stage = createStage(stage_id, last_stage, stage_root_node); +// } } size_t next_child = frame.visited_children; @@ -340,15 +340,15 @@ void QueryPlan::scheduleStages(ContextPtr context) String my_replica = context->getMacros()->getValue("replica") + ":" + toString(context->getServerPort("grpc_port")); auto distributed_source_step = std::make_unique( header, parent_stage->executors, query_id, result_stage->id, parent_stage->id, my_replica, context); + reset(); + addStep(std::move(distributed_source_step)); { /// Only for debug. LOG_DEBUG( log, "Local plan fragment:\n{}", - debugLocalPlanFragment(query_id, result_stage->id, my_replica, distributed_source_step.get())); + debugLocalPlanFragment(query_id, result_stage->id, my_replica, std::vector{root})); } - reset(); - addStep(std::move(distributed_source_step)); continue; } @@ -365,8 +365,6 @@ void QueryPlan::scheduleStages(ContextPtr context) { LOG_DEBUG(log, "Plan fragment to send:\nquery: {}\n{}", context->getClientInfo().initial_query, debugRemotePlanFragment(*executor, query_id, &stage)); -// const auto & query_distributed_plan_info = Context::QueryPlanFragmentInfo{ -// .query_id = query_id, .stage_id = stage->id, .node_id = *executor, .sources = stage->sources, .sinks = stage->sinks}; GRPCQueryInfo query_info; query_info.set_query(context->getClientInfo().initial_query); query_info.set_query_id(query_id); @@ -374,8 +372,6 @@ void QueryPlan::scheduleStages(ContextPtr context) if (!stage.parents.empty()) query_info.set_parent_stage_id(stage.parents[0]->id); query_info.set_node_id(*executor); -// query_info.set_user_name(context->getUserName()); -// query_info.set_password(context->getClientInfo().) for (const auto & source : stage.sources) { query_info.add_sources(*source); @@ -394,13 +390,12 @@ void QueryPlan::scheduleStages(ContextPtr context) void QueryPlan::buildPlanFragment(ContextPtr context) { - LOG_DEBUG(log, "Building plan fragment."); + LOG_DEBUG(log, "Build plan fragment."); /// Get my replica grpc address String my_replica = context->getMacros()->getValue("replica") + ":" + toString(context->getServerPort("grpc_port")); const auto & query_distributed_plan_info = context->getQueryPlanFragmentInfo(); int my_stage_id = query_distributed_plan_info.stage_id; -// int parent_stage_id = query_distributed_plan_info.parent_stage_id; struct Frame { @@ -415,8 +410,8 @@ void QueryPlan::buildPlanFragment(ContextPtr context) /// Used for locating the plan fragment. int stage_id = -1; - Node * fragment_root_node = nullptr; - Node * fragment_leaf_node = nullptr; /// TODO: For join or union, there are more than one leaf nodes. + Node * child_node = nullptr; + std::vector distributed_source_nodes; /// Only for debug while (!stack.empty()) { @@ -424,53 +419,49 @@ void QueryPlan::buildPlanFragment(ContextPtr context) if (one_child_is_visited) { - ++frame.visited_children; - one_child_is_visited = false; - - /// TODO: This is a shuffle relationship between current node and the last visited child(i.e. fragment_root_node). -// if (true) + /// TODO: This is a shuffle dependency between current node and the last visited child. +// if (false) // { // stage_id++; -// if (stage_id == my_stage_id) +// /// Add a DistributedSourceStep between current node and child node. +// if (stage_id == query_distributed_plan_info.parent_stage_id) /// TODO: if(query_distributed_plan_info.parent_id_to_sources.contains(stage_id)) // { -// root = fragment_root_node; -// DistributedSourceStep * step = nullptr; -// if (fragment_leaf_node) -// { -// /// Set sources for fragment_leaf_node -// step = dynamic_cast(fragment_leaf_node->step.get()); -// step->setSources(query_distributed_plan_info.sources); -// } +// /// Create a DistributedSourceStep. +// const auto & header = child_node->step->getOutputStream().header; +// const String & query_id = context->getClientInfo().initial_query_id; +// const auto & sources = query_distributed_plan_info.sources; // TODO: query_distributed_plan_info.parent_id_to_sources[stage_id]; +// auto distributed_source_step = std::make_unique( +// header, sources, query_id, my_stage_id, stage_id, my_replica, context); // +// /// Reuse child node, but replace its step with DistributedSourceStep. +// assert(child_node == frame.node->children[frame.visited_children]); +// child_node->step = std::move(distributed_source_step); +// child_node->children.clear(); +// distributed_source_nodes.emplace_back(child_node); +// } +// else if (stage_id == my_stage_id) +// { +// root = child_node; // { // /// Only for debug. // LOG_DEBUG( // log, // "Local plan fragment:\n{}", -// debugLocalPlanFragment(query_distributed_plan_info.query_id, stage_id, my_replica, step)); +// debugLocalPlanFragment(query_distributed_plan_info.query_id, stage_id, my_replica, distributed_source_nodes)); // } // return; // } -// -// /// Add a DistributedSourceStep between current node and fragment_root_node. -// /// But this step may NOT be executed in my fragment. -// frame.node->children.clear(); -// const auto & header = fragment_root_node->step->getOutputStream().header; -// const String & query_id = context->getClientInfo().initial_query_id; -// auto distributed_source_step = std::make_unique(header, {}, query_id, my_stage_id, parent_stage_id, my_replica, context); -// insertStep(frame.node->step, distributed_source_step); -// nodes.emplace_back(Node{.step = std::move(step)}); -// fragment_leaf_node = &nodes.back(); -// frame.node->children.emplace_back(fragment_leaf_node); // } + + ++frame.visited_children; + one_child_is_visited = false; } size_t next_child = frame.visited_children; if (next_child == frame.node->children.size()) { LOG_DEBUG(log, "Visited step: {}", frame.node->step->getName()); - - fragment_root_node = frame.node; + child_node = frame.node; one_child_is_visited = true; stack.pop(); } @@ -482,19 +473,13 @@ void QueryPlan::buildPlanFragment(ContextPtr context) ++stage_id; if (stage_id == my_stage_id) { - root = fragment_root_node; - DistributedSourceStep * step = nullptr; - if (fragment_leaf_node) - { - step = dynamic_cast(fragment_leaf_node->step.get()); - step->setSources(query_distributed_plan_info.sources); - } + root = child_node; { /// Only for debug. LOG_DEBUG( log, "Local plan fragment:\n{}", - debugLocalPlanFragment(query_distributed_plan_info.query_id, stage_id, my_replica, step)); + debugLocalPlanFragment(query_distributed_plan_info.query_id, stage_id, my_replica, distributed_source_nodes)); } return; } @@ -535,18 +520,17 @@ void QueryPlan::buildDistributedPlan(ContextMutablePtr context) } } -String QueryPlan::debugLocalPlanFragment(const String & query_id, int stage_id, const String & node_id, const DistributedSourceStep * step) +String QueryPlan::debugLocalPlanFragment(const String & query_id, int stage_id, const String & node_id, const std::vector distributed_source_nodes) { WriteBufferFromOwnString buf; ExplainPlanOptions options; buf << "fragment id: " << query_id << "/" << stage_id << "/" << node_id << "\n"; - if (step) + for (const auto node : distributed_source_nodes) { - buf << "sources: "; - for (const auto & source : step->getSources()) - { + auto distributed_source_step = dynamic_cast(node->step.get()); + buf << distributed_source_step->getName() << ": "; + for (const auto & source : distributed_source_step->getSources()) buf << *source << " "; - } buf.write('\n'); } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 19088cd66b9b..31b3637a7bf9 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -136,7 +136,7 @@ class QueryPlan }; using PlanFragmentInfoPtr = std::shared_ptr; - String debugLocalPlanFragment(const String & query_id, int stage_id, const String & node_id, const DistributedSourceStep * stage); + String debugLocalPlanFragment(const String & query_id, int stage_id, const String & node_id, const std::vector distributed_source_nodes); String debugRemotePlanFragment(const String & receiver, const String & query_id, const Stage * stage); private: From ea7d64b188d1bf05358e13b8462ba2e93d2fd2c4 Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Thu, 9 Dec 2021 08:33:32 +0000 Subject: [PATCH 03/84] Support distributed ALTER UPDATE/DELETE statement. --- src/Interpreters/DDLTask.cpp | 6 ++--- src/Interpreters/InterpreterAlterQuery.cpp | 13 ++++++++-- src/Interpreters/InterpreterCreateQuery.cpp | 5 ++-- src/Interpreters/InterpreterDropQuery.cpp | 9 +++++-- src/Interpreters/executeDDLQueryOnCluster.cpp | 26 +++++++------------ 5 files changed, 31 insertions(+), 28 deletions(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 6594650e73de..84d921c63324 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -197,12 +197,10 @@ void DDLTask::setClusterInfo(ContextPtr context, Poco::Logger * log) throw Exception("Received unknown DDL query", ErrorCodes::UNKNOWN_TYPE_OF_QUERY); cluster_name = query_on_cluster->cluster; - /// If all cluster, ignore check + /// If cluster name is all, modify the name according to running mode if (cluster_name == CLUSTER_TYPE_ALL) { - query = query_on_cluster->getRewrittenASTWithoutOnCluster(address_in_cluster.default_database); - query_on_cluster = nullptr; - return; + cluster_name = context->getRunningMode() == Context::RunningMode::COMPUTE ? CLUSTER_TYPE_COMPUTE : CLUSTER_TYPE_STORE; } cluster = context->tryGetCluster(cluster_name); diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 2e4abf1a8393..bc6f5d8553de 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -134,6 +134,11 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) if (!mutation_commands.empty()) { + if (alter.is_initial) + { + const_cast(alter).cluster = CLUSTER_TYPE_STORE; + return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess()); + } table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate(); table->mutate(mutation_commands, getContext()); @@ -141,6 +146,11 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) if (!partition_commands.empty()) { + if (alter.is_initial) + { + const_cast(alter).cluster = CLUSTER_TYPE_STORE; + return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess()); + } table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, getContext()->getSettingsRef()); auto partition_commands_pipe = table->alterPartition(metadata_snapshot, partition_commands, getContext()); if (!partition_commands_pipe.empty()) @@ -204,9 +214,8 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) }; String meta_info = formatAlterCommandsToSql(zookeeper->get(path), getContext(), alter_commands); const_cast(alter).cluster = CLUSTER_TYPE_ALL; - executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess(), path, meta_info); + return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess(), path, meta_info); } - return {}; } if (!alter_commands.empty()) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 14b2feea40c9..a60626aa1f32 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1324,7 +1324,7 @@ BlockIO InterpreterCreateQuery::execute() } } - create.cluster = CLUSTER_TYPE_STORE; + create.cluster = getContext()->getRunningMode() == Context::RunningMode::COMPUTE ? CLUSTER_TYPE_COMPUTE : CLUSTER_TYPE_STORE; prepareOnClusterQuery(create, getContext(), create.cluster); ASTCreateQuery create_tmp = {create}; String meta_info; @@ -1368,8 +1368,7 @@ BlockIO InterpreterCreateQuery::execute() /// All cluster to execute create.cluster = CLUSTER_TYPE_ALL; LOG_DEBUG(log, "DDL query on cluster: {}, create {}.{}", create.cluster, create.database, create.table); - executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess(), path, meta_info); - return {}; + return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess(), path, meta_info); } if (!create.cluster.empty()) { diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index cf7cf4b5cb7a..16da54472213 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -77,8 +77,13 @@ BlockIO InterpreterDropQuery::execute() return {}; } drop.cluster = CLUSTER_TYPE_ALL; - executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccessForDDLOnCluster(), path, ""); - return {}; + drop.no_delay = true; + if (drop.kind == ASTDropQuery::Kind::Truncate) + { + drop.cluster = CLUSTER_TYPE_STORE; + path = ""; + } + return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccessForDDLOnCluster(), path, ""); } if (!drop.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccessForDDLOnCluster()); diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 7136b793542b..368222462499 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -85,11 +85,11 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, } } - bool is_all_cluster = query->cluster == CLUSTER_TYPE_ALL; + bool is_reserved_cluster = query->cluster == CLUSTER_TYPE_ALL || query->cluster == CLUSTER_TYPE_STORE || query->cluster == CLUSTER_TYPE_COMPUTE; DDLWorker & ddl_worker = context->getDDLWorker(); std::vector hosts; Cluster::AddressesWithFailover shards; - if (!is_all_cluster) + if (!is_reserved_cluster) { query->cluster = context->getMacros()->expand(query->cluster); ClusterPtr cluster = context->getCluster(query->cluster); @@ -103,22 +103,14 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, } else { - if (meta_path.empty()) - throw Exception("Empty meta path in a distributed DDL task", ErrorCodes::LOGICAL_ERROR); /// Get Hosts from meta service - auto getHostsFromMetaService = [&context](std::vector& hosts){ - auto zookeeper = context->getZooKeeper(); - auto clusters_path = DEFAULT_ZOOKEEPER_CLUSTERS_PATH; - Strings children = zookeeper->getChildren(clusters_path); - for(auto& child: children) + auto getHostsFromMetaService = [&context, &query](std::vector& hosts){ + std::unordered_map replicas = context->getClustersWatcher().getContainer(); + for (const auto & replica : replicas) { - auto host_port_str = zookeeper->get(fs::path(clusters_path) / child); - auto pos = host_port_str.find(':'); - if (pos == std::string::npos) - continue; - auto host = host_port_str.substr(0, pos); - HostID hostId(host, context->getTCPPort()); - hosts.emplace_back(std::move(hostId)); + const auto & replica_info = replica.second; + if (query->cluster == CLUSTER_TYPE_ALL || (query->cluster == replica_info->type)) + hosts.emplace_back(HostID(replica_info->name, context->getTCPPort())); } }; getHostsFromMetaService(hosts); @@ -137,7 +129,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, bool use_local_default_database = false; const String & current_database = context->getCurrentDatabase(); - if (!is_all_cluster && need_replace_current_database) + if (!is_reserved_cluster && need_replace_current_database) { Strings shard_default_databases; for (const auto & shard : shards) From 20bb20e932128dac889bbf9d1ef1161b9092d09d Mon Sep 17 00:00:00 2001 From: Chao Ma Date: Wed, 8 Dec 2021 20:02:21 +0800 Subject: [PATCH 04/84] Add insert distributed for replicate merge tree --- src/Core/Settings.h | 6 +- .../Distributed/DistributedSettings.h | 132 +++++++++++++++++- src/Storages/Distributed/DistributedSink.cpp | 25 ++-- .../MergeTree/registerStorageMergeTree.cpp | 8 +- src/Storages/StorageDistributed.cpp | 80 +++++++++-- src/Storages/StorageReplicatedMergeTree.cpp | 13 +- src/Storages/StorageReplicatedMergeTree.h | 5 +- src/Storages/registerStorages.cpp | 2 + 8 files changed, 240 insertions(+), 31 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e6f685a06502..ace2ccb9dcd6 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -234,7 +234,7 @@ class IColumn; M(Bool, fallback_to_stale_replicas_for_distributed_queries, true, "Suppose max_replica_delay_for_distributed_queries is set and all replicas for the queried table are stale. If this setting is enabled, the query will be performed anyway, otherwise the error will be reported.", 0) \ M(UInt64, preferred_max_column_in_block_size_bytes, 0, "Limit on max column size in block while reading. Helps to decrease cache misses count. Should be close to L2 cache size.", 0) \ \ - M(Bool, insert_distributed_sync, false, "If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster.", 0) \ + M(Bool, insert_distributed_sync, true, "If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster.", 0) \ M(UInt64, insert_distributed_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) \ M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite. Zero means async mode.", 0) \ M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ @@ -375,7 +375,7 @@ class IColumn; M(Bool, low_cardinality_use_single_dictionary_for_part, false, "LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.", 0) \ M(Bool, decimal_check_overflow, true, "Check overflow of decimal arithmetic/comparison operations", 0) \ \ - M(Bool, prefer_localhost_replica, true, "If it's true then queries will be always sent to local replica (if it exists). If it's false then replica to send a query will be chosen between local and remote ones according to load_balancing", 0) \ + M(Bool, prefer_localhost_replica, false, "If it's true then queries will be always sent to local replica (if it exists). If it's false then replica to send a query will be chosen between local and remote ones according to load_balancing", 0) \ M(UInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.", 0) \ M(UInt64, http_max_multipart_form_data_size, 1024 * 1024 * 1024, "Limit on size of multipart/form-data content. This setting cannot be parsed from URL parameters and should be set in user profile. Note that content is parsed and external tables are created in memory before start of query execution. And this is the only limit that has effect on that stage (limits on max memory usage and max execution time have no effect while reading HTTP form data).", 0) \ M(Bool, calculate_text_stack_trace, true, "Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when huge amount of wrong queries are executed. In normal cases you should not disable this option.", 0) \ @@ -621,7 +621,7 @@ class IColumn; M(Bool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \ M(Bool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \ M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) \ - M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ + M(Bool, insert_distributed_one_random_shard, true, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ \ M(Bool, cross_to_inner_join_rewrite, true, "Use inner join instead of comma/cross join if possible", 0) \ \ diff --git a/src/Storages/Distributed/DistributedSettings.h b/src/Storages/Distributed/DistributedSettings.h index 8cc942cab027..db7dd03ef947 100644 --- a/src/Storages/Distributed/DistributedSettings.h +++ b/src/Storages/Distributed/DistributedSettings.h @@ -15,18 +15,144 @@ namespace DB class ASTStorage; #define LIST_OF_DISTRIBUTED_SETTINGS(M) \ - M(Bool, fsync_after_insert, false, "Do fsync for every inserted. Will decreases performance of inserts (only for async INSERT, i.e. insert_distributed_sync=false)", 0) \ + M(Bool, embedded_fsync_after_insert, false, "Do fsync for every inserted. Will decreases performance of inserts (only for async INSERT, i.e. insert_distributed_sync=false)", 0) \ M(Bool, fsync_directories, false, "Do fsync for temporary directory (that is used for async INSERT only) after all part operations (writes, renames, etc.).", 0) \ /** Inserts settings. */ \ M(UInt64, bytes_to_throw_insert, 0, "If more than this number of compressed bytes will be pending for async INSERT, an exception will be thrown. 0 - do not throw.", 0) \ M(UInt64, bytes_to_delay_insert, 0, "If more than this number of compressed bytes will be pending for async INSERT, the query will be delayed. 0 - do not delay.", 0) \ - M(UInt64, max_delay_to_insert, 60, "Max delay of inserting data into Distributed table in seconds, if there are a lot of pending bytes for async send.", 0) \ + M(UInt64, embedded_max_delay_to_insert, 60, "Max delay of inserting data into Distributed table in seconds, if there are a lot of pending bytes for async send.", 0) \ /** Directory monitor settings */ \ M(UInt64, monitor_batch_inserts, 0, "Default - distributed_directory_monitor_batch_inserts", 0) \ M(UInt64, monitor_split_batch_on_failure, 0, "Default - distributed_directory_monitor_split_batch_on_failure", 0) \ M(Milliseconds, monitor_sleep_time_ms, 0, "Default - distributed_directory_monitor_sleep_time_ms", 0) \ M(Milliseconds, monitor_max_sleep_time_ms, 0, "Default - distributed_directory_monitor_max_sleep_time_ms", 0) \ - + \ + \ + \ + /** MergeTree settings */ \ + M(UInt64, min_compress_block_size, 0, "When granule is written, compress the data in buffer if the size of pending uncompressed data is larger or equal than the specified threshold. If this setting is not set, the corresponding global setting is used.", 0) \ + M(UInt64, max_compress_block_size, 0, "Compress the pending uncompressed data in buffer if its size is larger or equal than the specified threshold. Block of data will be compressed even if the current granule is not finished. If this setting is not set, the corresponding global setting is used.", 0) \ + M(UInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ + \ + /** Data storing format settings. */ \ + M(UInt64, min_bytes_for_wide_part, 10485760, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ + M(UInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ + M(UInt64, min_bytes_for_compact_part, 0, "Experimental. Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \ + M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ + M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ + M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ + \ + /** Merge settings. */ \ + M(UInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \ + M(UInt64, max_bytes_to_merge_at_max_space_in_pool, 150ULL * 1024 * 1024 * 1024, "Maximum in total size of parts to merge, when there are maximum free threads in background pool (or entries in replication queue).", 0) \ + M(UInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \ + M(UInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + M(UInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + M(UInt64, max_replicated_merges_with_ttl_in_queue, 1, "How many tasks of merging parts with TTL are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ + M(UInt64, number_of_free_entries_in_pool_to_execute_mutation, 20, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + M(UInt64, max_number_of_merges_with_ttl_in_pool, 2, "When there is more than specified number of merges with TTL entries in pool, do not assign new merge with TTL. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + M(Seconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ + M(Seconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories. You should not lower this value because merges and mutations may not be able to work with low value of this setting.", 0) \ + M(Seconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ + M(UInt64, min_rows_to_fsync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ + M(UInt64, min_compressed_bytes_to_fsync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + M(UInt64, min_compressed_bytes_to_fsync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ + M(Bool, fsync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ + M(Bool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ + M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Amount of bytes, accumulated in WAL to do fsync.", 0) \ + M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Interval in milliseconds after which fsync for WAL is being done.", 0) \ + M(Bool, in_memory_parts_insert_sync, false, "If true insert of part with in-memory format will wait for fsync of WAL", 0) \ + M(UInt64, non_replicated_deduplication_window, 0, "How many last blocks of hashes should be kept on disk (0 - disabled).", 0) \ + M(UInt64, max_parts_to_merge_at_once, 100, "Max amount of parts which can be merged at once (0 - disabled). Doesn't affect OPTIMIZE FINAL query.", 0) \ + M(UInt64, merge_selecting_sleep_ms, 5000, "Sleep time for merge selecting when no part selected, a lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ + \ + /** Inserts settings. */ \ + M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ + M(UInt64, inactive_parts_to_delay_insert, 0, "If table contains at least that many inactive parts in single partition, artificially slow down insert into table.", 0) \ + M(UInt64, parts_to_throw_insert, 300, "If more than this number active parts in single partition, throw 'Too many parts ...' exception.", 0) \ + M(UInt64, inactive_parts_to_throw_insert, 0, "If more than this number inactive parts in single partition, throw 'Too many inactive parts ...' exception.", 0) \ + M(UInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.", 0) \ + M(UInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.", 0) \ + \ + /** Replication settings. */ \ + M(UInt64, replicated_deduplication_window, 100, "How many last blocks of hashes should be kept in ZooKeeper (old blocks will be deleted).", 0) \ + M(UInt64, replicated_deduplication_window_seconds, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ + M(UInt64, max_replicated_logs_to_keep, 1000, "How many records may be in log, if there is inactive replica. Inactive replica becomes lost when when this number exceed.", 0) \ + M(UInt64, min_replicated_logs_to_keep, 10, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.", 0) \ + M(Seconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ + M(UInt64, prefer_fetch_merged_part_size_threshold, 10ULL * 1024 * 1024 * 1024, "If sum size of parts exceeds this threshold and time passed after replication log entry creation is greater than \"prefer_fetch_merged_part_time_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ + M(Seconds, execute_merges_on_single_replica_time_threshold, 0, "When greater than zero only a single replica starts the merge immediately, others wait up to that amount of time to download the result instead of doing merges locally. If the chosen replica doesn't finish the merge during that amount of time, fallback to standard behavior happens.", 0) \ + M(Seconds, remote_fs_execute_merges_on_single_replica_time_threshold, 3 * 60 * 60, "When greater than zero only a single replica starts the merge immediatelys when merged part on shared storage and 'allow_remote_fs_zero_copy_replication' is enabled.", 0) \ + M(Seconds, try_fetch_recompressed_part_timeout, 7200, "Recompression works slow in most cases, so we don't start merge with recompression until this timeout and trying to fetch recompressed part from replica which assigned this merge with recompression.", 0) \ + M(Bool, always_fetch_merged_part, false, "If true, replica never merge parts and always download merged parts from other replicas.", 0) \ + M(UInt64, max_suspicious_broken_parts, 10, "Max broken parts, if more - deny automatic deletion.", 0) \ + M(UInt64, max_suspicious_broken_parts_bytes, 1ULL * 1024 * 1024 * 1024, "Max size of all broken parts, if more - deny automatic deletion.", 0) \ + M(UInt64, max_files_to_modify_in_alter_columns, 75, "Not apply ALTER if number of files for modification(deletion, addition) more than this.", 0) \ + M(UInt64, max_files_to_remove_in_alter_columns, 50, "Not apply ALTER, if number of files for deletion more than this.", 0) \ + M(Float, replicated_max_ratio_of_wrong_parts, 0.5, "If ratio of wrong parts to total number of parts is less than this - allow to start.", 0) \ + M(UInt64, replicated_max_parallel_fetches_for_host, DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT, "Limit parallel fetches from endpoint (actually pool size).", 0) \ + M(Seconds, replicated_fetches_http_connection_timeout, 0, "HTTP connection timeout for part fetch requests. Inherited from default profile `http_connection_timeout` if not set explicitly.", 0) \ + M(Seconds, replicated_fetches_http_send_timeout, 0, "HTTP send timeout for part fetch requests. Inherited from default profile `http_send_timeout` if not set explicitly.", 0) \ + M(Seconds, replicated_fetches_http_receive_timeout, 0, "HTTP receive timeout for fetch part requests. Inherited from default profile `http_receive_timeout` if not set explicitly.", 0) \ + M(Bool, replicated_can_become_leader, true, "If true, Replicated tables replicas on this node will try to acquire leadership.", 0) \ + M(Seconds, zookeeper_session_expiration_check_period, 60, "ZooKeeper session expiration check period, in seconds.", 0) \ + M(Bool, detach_old_local_parts_when_cloning_replica, true, "Do not remove old local parts when repairing lost replica.", 0) \ + M(Bool, detach_not_byte_identical_parts, false, "Do not remove non byte-idential parts for ReplicatedMergeTree, instead detach them (maybe useful for further analysis).", 0) \ + M(UInt64, max_replicated_fetches_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ + M(UInt64, max_replicated_sends_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ + \ + /** Check delay of replicas settings. */ \ + M(UInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \ + M(UInt64, cleanup_delay_period, 30, "Period to clean old queue logs, blocks hashes and parts.", 0) \ + M(UInt64, cleanup_delay_period_random_add, 10, "Add uniformly distributed value from 0 to x seconds to cleanup_delay_period to avoid thundering herd effect and subsequent DoS of ZooKeeper in case of very large number of tables.", 0) \ + M(UInt64, min_relative_delay_to_close, 300, "Minimal delay from other replicas to close, stop serving requests and not return Ok during status check.", 0) \ + M(UInt64, min_absolute_delay_to_close, 0, "Minimal absolute delay to close, stop serving requests and not return Ok during status check.", 0) \ + M(UInt64, enable_vertical_merge_algorithm, 1, "Enable usage of Vertical merge algorithm.", 0) \ + M(UInt64, vertical_merge_algorithm_min_rows_to_activate, 16 * DEFAULT_MERGE_BLOCK_SIZE, "Minimal (approximate) sum of rows in merging parts to activate Vertical merge algorithm.", 0) \ + M(UInt64, vertical_merge_algorithm_min_columns_to_activate, 11, "Minimal amount of non-PK columns to activate Vertical merge algorithm.", 0) \ + \ + /** Compatibility settings */ \ + M(Bool, compatibility_allow_sampling_expression_not_in_primary_key, false, "Allow to create a table with sampling expression not in primary key. This is needed only to temporarily allow to run the server with wrong tables for backward compatibility.", 0) \ + M(Bool, use_minimalistic_checksums_in_zookeeper, true, "Use small format (dozens bytes) for part checksums in ZooKeeper instead of ordinary ones (dozens KB). Before enabling check that all replicas support new format.", 0) \ + M(Bool, use_minimalistic_part_header_in_zookeeper, true, "Store part header (checksums and columns) in a compact format and a single part znode instead of separate znodes (/columns and /checksums). This can dramatically reduce snapshot size in ZooKeeper. Before enabling check that all replicas support new format.", 0) \ + M(UInt64, finished_mutations_to_keep, 100, "How many records about mutations that are done to keep. If zero, then keep all of them.", 0) \ + M(UInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).", 0) \ + M(UInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).", 0) \ + M(UInt64, min_index_granularity_bytes, 1024, "Minimum amount of bytes in single granule.", 1024) \ + M(Int64, merge_with_ttl_timeout, 3600 * 4, "Minimal time in seconds, when merge with delete TTL can be repeated.", 0) \ + M(Int64, merge_with_recompression_ttl_timeout, 3600 * 4, "Minimal time in seconds, when merge with recompression TTL can be repeated.", 0) \ + M(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ + M(Bool, materialize_ttl_recalculate_only, false, "Only recalculate ttl info when MATERIALIZE TTL", 0) \ + M(Bool, enable_mixed_granularity_parts, true, "Enable parts with adaptive and non adaptive granularity", 0) \ + M(MaxThreads, max_part_loading_threads, 0, "The number of threads to load data parts at startup.", 0) \ + M(MaxThreads, max_part_removal_threads, 0, "The number of threads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).", 0) \ + M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ + M(String, storage_policy, "default", "Name of storage disk policy", 0) \ + M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ + M(Bool, allow_remote_fs_zero_copy_replication, true, "Allow Zero-copy replication over remote fs", 0) \ + M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm", 0) \ + M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \ + M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \ + M(UInt64, max_concurrent_queries, 0, "Max number of concurrently executed queries related to the MergeTree table (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ + M(UInt64, min_marks_to_honor_max_concurrent_queries, 0, "Minimal number of marks to honor the MergeTree-level's max_concurrent_queries (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ + M(UInt64, min_bytes_to_rebalance_partition_over_jbod, 0, "Minimal amount of bytes to enable part rebalance over JBOD array (0 - disabled).", 0) \ + M(Bool, check_sample_column_is_correct, true, "Check columns or columns by hash for sampling are unsigned integer.", 0) \ + \ + /** Experimental/work in progress feature. Unsafe for production. */ \ + M(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", 0) \ + M(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ + \ + /** Obsolete settings. Kept for backward compatibility only. */ \ + M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \ + M(UInt64, check_delay_period, 60, "Obsolete setting, does nothing.", 0) \ + M(Bool, allow_floating_point_partition_key, false, "Allow floating point as partition key", 0) \ + M(UInt64, replicated_max_parallel_sends, 0, "Obsolete setting, does nothing.", 0) \ + M(UInt64, replicated_max_parallel_sends_for_table, 0, "Obsolete setting, does nothing.", 0) \ + M(UInt64, replicated_max_parallel_fetches, 0, "Obsolete setting, does nothing.", 0) \ + M(UInt64, replicated_max_parallel_fetches_for_table, 0, "Obsolete setting, does nothing.", 0) \ + M(Bool, write_final_mark, true, "Obsolete setting, does nothing.", 0) +/// Settings that should not change after the creation of a table. DECLARE_SETTINGS_TRAITS(DistributedSettingsTraits, LIST_OF_DISTRIBUTED_SETTINGS) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index cf349d1f8cf8..39857a307865 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -133,7 +133,7 @@ DistributedSink::DistributedSink( if (settings.max_distributed_depth && context->getClientInfo().distributed_depth > settings.max_distributed_depth) throw Exception("Maximum distributed depth exceeded", ErrorCodes::TOO_LARGE_DISTRIBUTED_DEPTH); context->getClientInfo().distributed_depth += 1; - random_shard_insert = settings.insert_distributed_one_random_shard && !storage.has_sharding_key; + random_shard_insert = settings.insert_distributed_one_random_shard; } @@ -168,20 +168,23 @@ Block DistributedSink::removeSuperfluousColumns(Block block) const void DistributedSink::writeAsync(const Block & block) { - if (random_shard_insert) + const Settings & settings = context->getSettingsRef(); + + if (settings.insert_shard_id) { - writeAsyncImpl(block, storage.getRandomShardIndex(cluster->getShardsInfo())); + writeAsyncImpl(block, settings.insert_shard_id - 1); ++inserted_blocks; + return; } - else - { - if (storage.getShardingKeyExpr() && (cluster->getShardsInfo().size() > 1)) - return writeSplitAsync(block); - - writeAsyncImpl(block); + if (random_shard_insert) + { + writeAsyncImpl(block, storage.getRandomShardIndex(cluster->getShardsInfo())); ++inserted_blocks; + return; } + + LOG_WARNING(log, "Could not be happened, settings without insert_shard_id and random_shard_insert"); } @@ -453,7 +456,7 @@ void DistributedSink::writeSync(const Block & block) watch_current_block.restart(); - if (random_shard_insert) + if (!settings.insert_shard_id && random_shard_insert) { start = storage.getRandomShardIndex(shards_info); end = start + 1; @@ -655,7 +658,7 @@ void DistributedSink::writeToShard(const Block & block, const std::vectorgetSettingsRef(); const auto & distributed_settings = storage.getDistributedSettingsRef(); - bool fsync = distributed_settings.fsync_after_insert; + bool fsync = distributed_settings.embedded_fsync_after_insert; bool dir_fsync = distributed_settings.fsync_directories; std::string compression_method = Poco::toUpper(settings.network_compression_method.toString()); diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 982acfe62a43..2b7098b8a3e3 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -792,6 +792,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (arg_num != arg_cnt) throw Exception("Wrong number of engine arguments.", ErrorCodes::BAD_ARGUMENTS); + StoragePtr embedded_distributed; + if (args.getContext()->getRunningMode() == Context::RunningMode::COMPUTE) { + embedded_distributed = StorageFactory::instance().getAllStorages().at("EmbeddedDistributed").creator_fn(args); + } + if (replicated) return StorageReplicatedMergeTree::create( zookeeper_path, @@ -805,7 +810,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) merging_params, std::move(storage_settings), args.has_force_restore_data_flag, - allow_renaming); + allow_renaming, + embedded_distributed); else return StorageMergeTree::create( args.table_id, diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 91d54f0d24af..65ecf0c2496f 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -647,11 +647,11 @@ SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadata auto shard_num = cluster->getLocalShardCount() + cluster->getRemoteShardCount(); - /// If sharding key is not specified, then you can only write to a shard containing only one shard - if (!settings.insert_shard_id && !settings.insert_distributed_one_random_shard && !has_sharding_key && shard_num >= 2) + /// Without random and specified shard id are not allow + if (!settings.insert_shard_id && !settings.insert_distributed_one_random_shard && shard_num >= 2) { throw Exception( - "Method write is not supported by storage " + getName() + " with more than one shard and no sharding key provided", + "Method write is not supported by storage " + getName() + " with more than one shard, no insert_shard_id and insert_distributed_one_random_shard=true provided", ErrorCodes::STORAGE_REQUIRES_PARAMETER); } @@ -660,8 +660,7 @@ SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadata throw Exception("Shard id should be range from 1 to shard number", ErrorCodes::INVALID_SHARD_ID); } - /// Force sync insertion if it is remote() table function - bool insert_sync = settings.insert_distributed_sync || settings.insert_shard_id || owned_cluster; + bool insert_sync = settings.insert_distributed_sync || owned_cluster; auto timeout = settings.insert_distributed_timeout; Names columns_to_send; @@ -1242,14 +1241,14 @@ void StorageDistributed::delayInsertOrThrowIfNeeded() const if (distributed_settings.bytes_to_delay_insert && total_bytes > distributed_settings.bytes_to_delay_insert) { /// Step is 5% of the delay and minimal one second. - /// NOTE: max_delay_to_insert is in seconds, and step is in ms. - const size_t step_ms = std::min(1., double(distributed_settings.max_delay_to_insert) * 1'000 * 0.05); + /// NOTE: embedded_max_delay_to_insert is in seconds, and step is in ms. + const size_t step_ms = std::min(1., double(distributed_settings.embedded_max_delay_to_insert) * 1'000 * 0.05); UInt64 delayed_ms = 0; do { delayed_ms += step_ms; std::this_thread::sleep_for(std::chrono::milliseconds(step_ms)); - } while (*totalBytes(getContext()->getSettingsRef()) > distributed_settings.bytes_to_delay_insert && delayed_ms < distributed_settings.max_delay_to_insert*1000); + } while (*totalBytes(getContext()->getSettingsRef()) > distributed_settings.bytes_to_delay_insert && delayed_ms < distributed_settings.embedded_max_delay_to_insert*1000); ProfileEvents::increment(ProfileEvents::DistributedDelayedInserts); ProfileEvents::increment(ProfileEvents::DistributedDelayedInsertsMilliseconds, delayed_ms); @@ -1344,9 +1343,9 @@ void registerStorageDistributed(StorageFactory & factory) distributed_settings.loadFromQuery(*args.storage_def); } - if (distributed_settings.max_delay_to_insert < 1) + if (distributed_settings.embedded_max_delay_to_insert < 1) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, - "max_delay_to_insert cannot be less then 1"); + "embedded_max_delay_to_insert cannot be less then 1"); if (distributed_settings.bytes_to_throw_insert && distributed_settings.bytes_to_delay_insert && distributed_settings.bytes_to_throw_insert <= distributed_settings.bytes_to_delay_insert) @@ -1387,4 +1386,65 @@ void registerStorageDistributed(StorageFactory & factory) }); } +void registerStorageEmbeddedDistributed(StorageFactory & factory) +{ + factory.registerStorage( + "EmbeddedDistributed", + [](const StorageFactory::Arguments & args) { + const ContextPtr & context = args.getContext(); + + /// TODO: move some arguments from the arguments to the SETTINGS. + DistributedSettings distributed_settings; + + if (args.storage_def->settings) + { + distributed_settings.loadFromQuery(*args.storage_def); + } + + if (distributed_settings.embedded_max_delay_to_insert < 1) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "embedded_max_delay_to_insert cannot be less then 1"); + + if (distributed_settings.bytes_to_throw_insert && distributed_settings.bytes_to_delay_insert + && distributed_settings.bytes_to_throw_insert <= distributed_settings.bytes_to_delay_insert) + { + throw Exception( + ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "bytes_to_throw_insert cannot be less or equal to bytes_to_delay_insert (since it is handled first)"); + } + + /// Set default values from the distributed_directory_monitor_* global context settings. + if (!distributed_settings.monitor_batch_inserts.changed) + distributed_settings.monitor_batch_inserts = context->getSettingsRef().distributed_directory_monitor_batch_inserts; + if (!distributed_settings.monitor_split_batch_on_failure.changed) + distributed_settings.monitor_split_batch_on_failure + = context->getSettingsRef().distributed_directory_monitor_split_batch_on_failure; + if (!distributed_settings.monitor_sleep_time_ms.changed) + distributed_settings.monitor_sleep_time_ms + = Poco::Timespan(context->getSettingsRef().distributed_directory_monitor_sleep_time_ms); + if (!distributed_settings.monitor_max_sleep_time_ms.changed) + distributed_settings.monitor_max_sleep_time_ms + = Poco::Timespan(context->getSettingsRef().distributed_directory_monitor_max_sleep_time_ms); + + return StorageDistributed::create( + args.table_id, + args.columns, + args.constraints, + args.comment, + args.table_id.database_name, + args.table_id.table_name, + "store", + context, + nullptr, + "default", + args.relative_data_path, + distributed_settings, + args.attach); + }, + { + .supports_settings = true, + .supports_parallel_insert = true, + .source_access_type = AccessType::REMOTE, + }); +} + } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f4a50f2e5531..24911bb0269a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -274,7 +274,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const MergingParams & merging_params_, std::unique_ptr settings_, bool has_force_restore_data_flag, - bool allow_renaming_) + bool allow_renaming_, + StoragePtr embedded_distributed_) : MergeTreeData(table_id_, relative_data_path_, metadata_, @@ -305,6 +306,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , replicated_fetches_pool_size(getContext()->getSettingsRef().background_fetches_pool_size) , replicated_fetches_throttler(std::make_shared(getSettings()->max_replicated_fetches_network_bandwidth, getContext()->getReplicatedFetchesThrottler())) , replicated_sends_throttler(std::make_shared(getSettings()->max_replicated_sends_network_bandwidth, getContext()->getReplicatedSendsThrottler())) + , embedded_distributed(embedded_distributed_) { queue_updating_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); }); @@ -4315,6 +4317,11 @@ void StorageReplicatedMergeTree::assertNotReadonly() const SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { + if (getContext()->getRunningMode() == Context::RunningMode::COMPUTE) + { + return embedded_distributed->write(nullptr, metadata_snapshot, local_context); + } + const auto storage_settings_ptr = getSettings(); assertNotReadonly(); @@ -4323,7 +4330,9 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con // TODO: should we also somehow pass list of columns to deduplicate on to the ReplicatedMergeTreeBlockOutputStream ? return std::make_shared( - *this, metadata_snapshot, query_settings.insert_quorum, + *this, + metadata_snapshot, + query_settings.insert_quorum, query_settings.insert_quorum_timeout.totalMilliseconds(), query_settings.max_partitions_per_insert_block, query_settings.insert_quorum_parallel, diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 65daf82a6338..cd556fe9e5db 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -393,6 +393,8 @@ class StorageReplicatedMergeTree final : public shared_ptr_helper void foreachCommittedParts(Func && func, bool select_sequential_consistency) const; @@ -743,7 +745,8 @@ class StorageReplicatedMergeTree final : public shared_ptr_helper settings_, bool has_force_restore_data_flag, - bool allow_renaming_); + bool allow_renaming_, + StoragePtr embedded_distributed_); }; String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info); diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index adbb51a37323..149a60f8cddc 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -14,6 +14,7 @@ void registerStorageNull(StorageFactory & factory); void registerStorageMerge(StorageFactory & factory); void registerStorageBuffer(StorageFactory & factory); void registerStorageDistributed(StorageFactory & factory); +void registerStorageEmbeddedDistributed(StorageFactory & factory); void registerStorageMemory(StorageFactory & factory); void registerStorageFile(StorageFactory & factory); void registerStorageURL(StorageFactory & factory); @@ -85,6 +86,7 @@ void registerStorages() registerStorageMerge(factory); registerStorageBuffer(factory); registerStorageDistributed(factory); + registerStorageEmbeddedDistributed(factory); registerStorageMemory(factory); registerStorageFile(factory); registerStorageURL(factory); From 89468c4c40187713a9cee33edbd1afcb25263780 Mon Sep 17 00:00:00 2001 From: madianjun Date: Tue, 7 Dec 2021 19:57:22 +0800 Subject: [PATCH 05/84] 1.Optimize the execution of plan fragment. 2.Fix "insert into ... select" to distributed plan. --- src/Client/GRPCClient.cpp | 74 +-- src/Client/GRPCClient.h | 20 +- src/Interpreters/Context.h | 6 +- .../IInterpreterUnionOrSelectQuery.cpp | 6 +- .../QueryPlan/DistributedSourceStep.h | 2 +- src/Processors/QueryPlan/QueryPlan.cpp | 36 +- src/Processors/QueryPlan/QueryPlan.h | 2 +- .../DistributedSourceExecutor.cpp | 4 +- src/QueryPipeline/DistributedSourceExecutor.h | 1 - src/Server/GRPCServer.cpp | 529 ++++++++++++++---- src/Server/grpc_protos/clickhouse_grpc.proto | 4 +- 11 files changed, 477 insertions(+), 207 deletions(-) diff --git a/src/Client/GRPCClient.cpp b/src/Client/GRPCClient.cpp index 633a4f79fcfe..efd743dac280 100644 --- a/src/Client/GRPCClient.cpp +++ b/src/Client/GRPCClient.cpp @@ -27,95 +27,65 @@ namespace ErrorCodes extern const int GRPC_READ_ERROR; } -GRPCClient::GRPCClient(const String & _addr) +GRPCClient::GRPCClient(const String & addr_) { - addr = _addr; - - log = &Poco::Logger::get("GRPCClient"); -} - -GRPCClient::~GRPCClient() -{ - reader_map.clear(); + addr = addr_; + log = &Poco::Logger::get("GRPCClient(" + addr + ")"); } -GRPCResult GRPCClient::SendDistributedPlanParams(GRPCQueryInfo & gQueryInfo) +GRPCResult GRPCClient::executePlanFragment(GRPCQueryInfo & query_info) { auto ch = grpc::CreateChannel(addr, grpc::InsecureChannelCredentials()); auto stub = clickhouse::grpc::ClickHouse::NewStub(ch); grpc::ClientContext ctx; GRPCResult result; /// Set to native format, cause we decode result by NativeReader in the read function - gQueryInfo.set_output_format("Native"); - grpc::Status status = stub->SendDistributedPlanParams(&ctx, gQueryInfo, &result); + query_info.set_output_format("Native"); + grpc::Status status = stub->ExecutePlanFragment(&ctx, query_info, &result); if (status.ok()) return result; else { LOG_ERROR( - log, "SendDistributedPlanParams to {} failed with code {}, query_id: {}", addr, status.error_code(), gQueryInfo.query_id()); + log, "Send query info to {} failed, code: {}, plan fragment id: {}.", addr, status.error_code(), query_info.query_id() + toString(query_info.stage_id()) + query_info.node_id()); throw Exception(status.error_message() + ", " + result.exception().display_text(), ErrorCodes::INVALID_GRPC_QUERY_INFO, true); } } -Block GRPCClient::read(const GRPCTicket & ticket) +void GRPCClient::prepareRead(const GRPCTicket & ticket) { - String key = ticket.query_id() + "/" + std::to_string(ticket.stage_id()) + "/" + ticket.node_id(); - std::shared_ptr inner_ctx; - - std::shared_lock lock(mu); - auto reader_it = reader_map.find(key); - if (reader_it != reader_map.end()) - inner_ctx = reader_it->second; - lock.unlock(); - - if (reader_it == reader_map.end()) { - std::unique_lock wlock(mu); - /// Check again - reader_it = reader_map.find(key); - if (reader_it != reader_map.end()) - inner_ctx = reader_it->second; - else - { - auto ch = grpc::CreateChannel(addr, grpc::InsecureChannelCredentials()); - std::shared_ptr ctx = std::make_shared(); - auto stub = clickhouse::grpc::ClickHouse::NewStub(ch); - auto cReader = stub->ExecuteQueryFragmentWithStreamOutput(ctx.get(), ticket); - reader_map[key] = std::make_shared(ch, ctx, stub, cReader); - reader_it = reader_map.find(key); - inner_ctx = reader_it->second; - } - wlock.unlock(); - } + auto ch = grpc::CreateChannel(addr, grpc::InsecureChannelCredentials()); + std::shared_ptr ctx = std::make_shared(); + auto stub = clickhouse::grpc::ClickHouse::NewStub(ch); + auto reader = stub->FetchPlanFragmentResult(ctx.get(), ticket); + inner_context = std::make_unique(ch, ctx, stub, reader); +} - LOG_DEBUG(log, "Read begin from {}", addr); +Block GRPCClient::read() +{ + assert(inner_context); + LOG_DEBUG(log, "Start reading result from {}.", addr); GRPCResult result; - if (reader_it->second->reader->Read(&result)) + if (inner_context->reader->Read(&result)) { - LOG_DEBUG(log, "Read result from {} success, exception.code: {}", addr, result.exception().code()); if (result.exception().code() != 0) { - LOG_ERROR(log, "GRPC addr: {} result exception: {} {}", addr, result.exception().code(), result.exception().display_text()); + LOG_ERROR(log, "Read from {} failed, exception.code: {}, exception.text: {}.", addr, result.exception().code(), result.exception().display_text()); throw Exception(result.exception().display_text(), ErrorCodes::INVALID_GRPC_QUERY_INFO, true); } - LOG_DEBUG(log, "Read from {} success, output size {}", addr, result.output().size()); - if (result.output().size() == 0) return {}; /// Read EOF ReadBufferFromString b(result.output()); NativeReader reader(b, 0 /* server_revision_ */); Block block = reader.read(); - LOG_DEBUG(log, "Read from {}, block decode success, has {} rows", addr, block.rows()); + LOG_DEBUG(log, "Read from {} success, result size: {}, block rows: {}.", addr, result.output().size(), block.rows()); return block; } - std::unique_lock wlock(mu); - reader_map.erase(reader_it); - wlock.unlock(); - throw Exception("read from grpc server failed! server: " + addr + ", " + std::to_string(result.exception().code()) + ", " + result.exception().display_text(), ErrorCodes::GRPC_READ_ERROR, true); + throw Exception("Read from grpc server " + addr + "failed, " + toString(result.exception().code()) + ", " + result.exception().display_text(), ErrorCodes::GRPC_READ_ERROR, true); } } diff --git a/src/Client/GRPCClient.h b/src/Client/GRPCClient.h index 73a7a1b2a6c5..7b6b4fd624de 100644 --- a/src/Client/GRPCClient.h +++ b/src/Client/GRPCClient.h @@ -26,15 +26,18 @@ using ReadDataCallback = std::function; class GRPCClient { public: - GRPCClient(const String & addr); - ~GRPCClient(); + GRPCClient(const String & addr_); + ~GRPCClient() = default; - /// Send distributed plan to other servers. - GRPCResult SendDistributedPlanParams(GRPCQueryInfo & g_query_info); + /// Send params of plan fragment to remote, and execute it. + GRPCResult executePlanFragment(GRPCQueryInfo & g_query_info); - /// Try to read a block from the remote server by the specified ticket, - /// If got EOF, an empty Block will be returned, you can use if (!block) to check it. - Block read(const GRPCTicket & ticket); + /// Initialize reader and inner context. + void prepareRead(const GRPCTicket & ticket); + + /// Try to read a block from remote. + /// If got EOF, an empty block will be returned, you can use if (!block) to check it. + Block read(); private: struct InnerContext @@ -57,8 +60,7 @@ class GRPCClient Poco::Logger * log; String addr; - std::map> reader_map; - mutable std::shared_mutex mu; + std::unique_ptr inner_context; }; } //#endif diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 4a5427a6136f..2f19e387400e 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -311,6 +311,7 @@ class Context: public std::enable_shared_from_this private: std::optional query_plan_fragment_info; /// It has no value if current node is initial compute node. + String select_query; using SampleBlockCache = std::unordered_map; mutable SampleBlockCache sample_block_cache; @@ -514,10 +515,13 @@ class Context: public std::enable_shared_from_this const QueryFactoriesInfo & getQueryFactoriesInfo() const { return query_factories_info; } void addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const; - bool isInitialComputeNode() const { return !query_plan_fragment_info; } + bool isInitialNode() const { return !query_plan_fragment_info; } const QueryPlanFragmentInfo & getQueryPlanFragmentInfo() const { return query_plan_fragment_info.value(); } void setQueryPlanFragmentInfo(const QueryPlanFragmentInfo & query_plan_fragment_info_) { query_plan_fragment_info = query_plan_fragment_info_; } + const String & getSelectQuery() const { return select_query; } + void setSelectQuery(const String & select_query_) { select_query = select_query_; } + StoragePtr executeTableFunction(const ASTPtr & table_expression); void addViewSource(const StoragePtr & storage); diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index 55c007e2713b..06fe1334094c 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -18,9 +19,12 @@ QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline() QueryPlan query_plan; buildQueryPlan(query_plan); + context->setSelectQuery(queryToString(this->query_ptr)); + query_plan.buildDistributedPlan(context); + QueryPlanOptimizationSettings do_not_optimize_plan{.optimize_plan = false}; return std::move(*query_plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context))); + do_not_optimize_plan, BuildQueryPipelineSettings::fromContext(context))); } } diff --git a/src/Processors/QueryPlan/DistributedSourceStep.h b/src/Processors/QueryPlan/DistributedSourceStep.h index 1dae70dbd933..f19c4fe6ffc0 100644 --- a/src/Processors/QueryPlan/DistributedSourceStep.h +++ b/src/Processors/QueryPlan/DistributedSourceStep.h @@ -17,7 +17,7 @@ class DistributedSourceStep final : public ISourceStep const String & node_id_, ContextPtr context_); - String getName() const override { return "DistributedSourceStep(" + toString(stage_id) + " -> " + toString(parent_stage_id); } + String getName() const override { return "DistributedSourceStep(" + toString(stage_id) + " <= " + toString(parent_stage_id) + ")"; } void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; const std::vector> & getSources() const { return sources; } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 5d07d6dfbb09..fd2ec1b01da5 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -363,10 +363,11 @@ void QueryPlan::scheduleStages(ContextPtr context) /// Send to each remote executor. for (const auto & executor : stage.executors) { - LOG_DEBUG(log, "Plan fragment to send:\nquery: {}\n{}", context->getClientInfo().initial_query, debugRemotePlanFragment(*executor, query_id, &stage)); - GRPCQueryInfo query_info; - query_info.set_query(context->getClientInfo().initial_query); + if (!context->getSelectQuery().empty()) + query_info.set_query(context->getSelectQuery()); + else + query_info.set_query(context->getClientInfo().initial_query); query_info.set_query_id(query_id); query_info.set_stage_id(stage.id); if (!stage.parents.empty()) @@ -380,10 +381,11 @@ void QueryPlan::scheduleStages(ContextPtr context) { query_info.add_sinks(*sink); } + LOG_DEBUG(log, "Plan fragment to send:\nquery: {}\n{}", query_info.query(), debugRemotePlanFragment(*executor, query_id, &stage)); GRPCClient cli(*executor); - auto result = cli.SendDistributedPlanParams(query_info); - LOG_INFO(log, "GRPCClient got result exception: {} {}.", result.exception().code(), result.exception().display_text()); + auto result = cli.executePlanFragment(query_info); + LOG_DEBUG(log, "GRPCClient got result, exception code: {}, exception text: {}.", result.exception().code(), result.exception().display_text()); } } } @@ -485,10 +487,14 @@ void QueryPlan::buildPlanFragment(ContextPtr context) } } -void QueryPlan::buildDistributedPlan(ContextMutablePtr context) +void QueryPlan::buildDistributedPlan(ContextPtr context) { - if (context->getRunningMode() == Context::RunningMode::STORE) + /// Query directly hit on the store node. + if (context->isInitialNode() && context->getRunningMode() == Context::RunningMode::STORE) + { + LOG_DEBUG(log, "Skip building distributed plan."); return; + } if (context->getInitialQueryId() == "zzzzzzzz-zzzz-zzzz-zzzz-zzzzzzzzzzzz") { @@ -498,17 +504,8 @@ void QueryPlan::buildDistributedPlan(ContextMutablePtr context) LOG_DEBUG(log, "Initial query id: {}, to be built to distributed plan.", context->getInitialQueryId()); -// /// For hard code debugging -// if (context->getMacros()->getValue("cluster") == "store") -// { -// auto query_plan_fragment_info -// = Context::QueryPlanFragmentInfo{.query_id = "xxx-yyy-zzz", .stage_id = 0, .node_id = "centos0"}; -// query_plan_fragment_info.sinks = {std::make_shared("ubuntu0")}; -// context->setQueryPlanFragmentInfo(query_plan_fragment_info); -// } - checkInitialized(); - if (context->isInitialComputeNode()) + if (context->isInitialNode()) { optimize(QueryPlanOptimizationSettings::fromContext(context)); buildStages(context); @@ -525,15 +522,16 @@ String QueryPlan::debugLocalPlanFragment(const String & query_id, int stage_id, WriteBufferFromOwnString buf; ExplainPlanOptions options; buf << "fragment id: " << query_id << "/" << stage_id << "/" << node_id << "\n"; + buf << "distributed source " << distributed_source_nodes.size() << " nodes:\n"; for (const auto node : distributed_source_nodes) { auto distributed_source_step = dynamic_cast(node->step.get()); - buf << distributed_source_step->getName() << ": "; + buf << distributed_source_step->getName() << ", sources: "; for (const auto & source : distributed_source_step->getSources()) buf << *source << " "; buf.write('\n'); } - + buf << "plan fragment:\n"; explainPlan(buf, options); return buf.str(); } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 31b3637a7bf9..9250089676c6 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -61,7 +61,7 @@ class QueryPlan void buildStages(ContextPtr context); /// Used by initial node. void scheduleStages(ContextPtr context); /// Used by initial node. void buildPlanFragment(ContextPtr context); /// Used by non-initial nodes. - void buildDistributedPlan(ContextMutablePtr context); + void buildDistributedPlan(ContextPtr context); QueryPipelineBuilderPtr buildQueryPipeline( const QueryPlanOptimizationSettings & optimization_settings, diff --git a/src/QueryPipeline/DistributedSourceExecutor.cpp b/src/QueryPipeline/DistributedSourceExecutor.cpp index 46488c070c2c..8ce34c0b84e2 100644 --- a/src/QueryPipeline/DistributedSourceExecutor.cpp +++ b/src/QueryPipeline/DistributedSourceExecutor.cpp @@ -17,9 +17,11 @@ DistributedSourceExecutor::DistributedSourceExecutor( , client(*source_) , log(&Poco::Logger::get("DistributedSourceExecutor(" + query_id + "/" + toString(stage_id) + "/" + node_id + ")")) { + GRPCTicket ticket; ticket.set_query_id(query_id); ticket.set_stage_id(stage_id); ticket.set_node_id(node_id); + client.prepareRead(ticket); } DistributedSourceExecutor::~DistributedSourceExecutor() @@ -33,7 +35,7 @@ Block DistributedSourceExecutor::read() try { - auto block = client.read(ticket); + auto block = client.read(); LOG_DEBUG(log, "Read block, rows: {}, columns: {}.", block.rows(), block.columns()); return block; } diff --git a/src/QueryPipeline/DistributedSourceExecutor.h b/src/QueryPipeline/DistributedSourceExecutor.h index 84c6adac1b1d..037dca893506 100644 --- a/src/QueryPipeline/DistributedSourceExecutor.h +++ b/src/QueryPipeline/DistributedSourceExecutor.h @@ -30,7 +30,6 @@ class DistributedSourceExecutor String query_id; String node_id; int stage_id; - GRPCTicket ticket; GRPCClient client; Poco::Logger * log = nullptr; std::atomic finished{false}; diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index ae659455ca54..cd6cb5659370 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -332,8 +332,8 @@ namespace CALL_WITH_STREAM_INPUT, /// ExecuteQueryWithStreamInput() call CALL_WITH_STREAM_OUTPUT, /// ExecuteQueryWithStreamOutput() call CALL_WITH_STREAM_IO, /// ExecuteQueryWithStreamIO() call - CALL_SEND_SIMPLE, /// SendDistributedPlanParams() call - CALL_FRAGMENT_WITH_STREAM_OUTPUT, /// ExecuteQueryFragmentWithStreamOutput() call + CALL_EXECUTE_PLAN_FRAGMENT, /// ExecutePlanFragment() call + CALL_FETCH_PLAN_FRAGMENT_RESULT, /// FetchPlanFragmentResult() call CALL_MAX, }; @@ -345,8 +345,8 @@ namespace case CALL_WITH_STREAM_INPUT: return "ExecuteQueryWithStreamInput()"; case CALL_WITH_STREAM_OUTPUT: return "ExecuteQueryWithStreamOutput()"; case CALL_WITH_STREAM_IO: return "ExecuteQueryWithStreamIO()"; - case CALL_SEND_SIMPLE: return "SendDistributedPlanParams()"; - case CALL_FRAGMENT_WITH_STREAM_OUTPUT: return "ExecuteQueryFragmentWithStreamOutput()"; + case CALL_EXECUTE_PLAN_FRAGMENT: return "ExecutePlanFragment()"; + case CALL_FETCH_PLAN_FRAGMENT_RESULT: return "FetchPlanFragmentResult()"; case CALL_MAX: break; } __builtin_unreachable(); @@ -359,7 +359,7 @@ namespace bool isOutputStreaming(CallType call_type) { - return (call_type == CALL_WITH_STREAM_OUTPUT) || (call_type == CALL_WITH_STREAM_IO) || (call_type == CALL_FRAGMENT_WITH_STREAM_OUTPUT); + return (call_type == CALL_WITH_STREAM_OUTPUT) || (call_type == CALL_WITH_STREAM_IO) || (call_type == CALL_FETCH_PLAN_FRAGMENT_RESULT); } template @@ -520,7 +520,7 @@ namespace }; template<> - class Responder : public BaseResponder + class Responder : public BaseResponder { public: void start(GRPCService & grpc_service, @@ -528,7 +528,7 @@ namespace grpc::ServerCompletionQueue & notification_queue, const CompletionCallback & callback) override { - grpc_service.RequestSendDistributedPlanParams(&grpc_context, &query_info.emplace(), &response_writer, &new_call_queue, ¬ification_queue, getCallbackPtr(callback)); + grpc_service.RequestExecutePlanFragment(&grpc_context, &query_info.emplace(), &response_writer, &new_call_queue, ¬ification_queue, getCallbackPtr(callback)); } void read(GRPCQueryInfo & query_info_, const CompletionCallback & callback) override @@ -561,7 +561,7 @@ namespace }; template<> - class Responder : public BaseResponder + class Responder : public BaseResponder { public: void start(GRPCService & grpc_service, @@ -569,7 +569,7 @@ namespace grpc::ServerCompletionQueue & notification_queue, const CompletionCallback & callback) override { - grpc_service.RequestExecuteQueryFragmentWithStreamOutput(&grpc_context, &ticket.emplace(), &writer, &new_call_queue, ¬ification_queue, getCallbackPtr(callback)); + grpc_service.RequestFetchPlanFragmentResult(&grpc_context, &ticket.emplace(), &writer, &new_call_queue, ¬ification_queue, getCallbackPtr(callback)); } void read(GRPCQueryInfo &, const CompletionCallback &) override @@ -609,8 +609,8 @@ namespace case CALL_WITH_STREAM_INPUT: return std::make_unique>(); case CALL_WITH_STREAM_OUTPUT: return std::make_unique>(); case CALL_WITH_STREAM_IO: return std::make_unique>(); - case CALL_SEND_SIMPLE: return std::make_unique>(); - case CALL_FRAGMENT_WITH_STREAM_OUTPUT: return std::make_unique>(); + case CALL_EXECUTE_PLAN_FRAGMENT: return std::make_unique>(); + case CALL_FETCH_PLAN_FRAGMENT_RESULT: return std::make_unique>(); case CALL_MAX: break; } __builtin_unreachable(); @@ -677,32 +677,170 @@ namespace class InnerMap { public: - bool getAndErase(const String & key_, Value & value_) + using Impl = std::unordered_map; + auto get(const String & key_) { std::lock_guard lock(mutex); auto it = container.find(key_); if (it == container.end()) - return false; - std::swap(it->second, value_); - container.erase(it); - return true; + return std::pair{{}, false}; + return std::pair{it->second, true}; } - bool insert(const String& key_, Value && value_) + auto insert(const String& key_, Value && value_) { std::lock_guard lock(mutex); - if (container.find(key_) != container.end()) - return false; - container.emplace(key_, value_); - return true; + return container.emplace(key_, value_); + } + + auto erase(const String & key_) + { + std::lock_guard lock(mutex); + return container.erase(key_); } + private: - std::unordered_map container; + Impl container; std::mutex mutex; }; - using QueryInfoMap = InnerMap; + class QueryInfoWrapper + { + public: + enum Status { + NORMAL, + TIMEOUT, + FINISH, + CANCEL + }; + + QueryInfoWrapper(GRPCQueryInfo * query_info_, int consumers_) + : query_info(query_info_), consumers(consumers_), blocks(consumers), ready(consumers, false), ready_count(0) + { + } + + void setWaitTimeoutSeconds(int wait_timeout_seconds_) + { + if (wait_timeout_seconds_ > 0) + wait_timeout_seconds = std::chrono::seconds(wait_timeout_seconds_); + } + + void notifyHeader(Block header_); + QueryInfoWrapper::Status waitConsume(); + void notifyReady(); + void notifyFinish(); + + QueryInfoWrapper::Status waitHeader(); + QueryInfoWrapper::Status waitReadyOrFinish(int index); + void notifyProduce(); + + GRPCQueryInfo * query_info; + int consumers; + + /// Transferred data between producer and consumer. + /// Not need to lock when accessing "blocks", because: + /// 1. producer will put blocks until all consumers take blocks out, so there is no READ when WRITE. + /// 2. consumers could take blocks simultaneously, as it's a vector which supports READ concurrently. + Block header; + std::vector blocks; + Block totals; + Block extremes; + ProfileInfo profile_info; + + /// Consumer wait condition. + std::mutex mutex_consumer; + std::condition_variable cv_consumer; + std::vector ready; + + /// Producer wait condition. + std::mutex mutex_producer; + std::condition_variable cv_producer; + int ready_count; + + std::atomic finish{false}; + std::atomic cancel{false}; + std::chrono::seconds wait_timeout_seconds{600}; + }; + + void QueryInfoWrapper::notifyHeader(Block header_) + { + { + std::lock_guard lock(mutex_consumer); + header = std::move(header_); + } + cv_consumer.notify_all(); + } + + QueryInfoWrapper::Status QueryInfoWrapper::waitConsume() + { + std::unique_lock lock(mutex_producer); + bool status = cv_producer.wait_for(lock, wait_timeout_seconds, [this] { return ready_count == 0 || cancel; }); + if (!status) + { + /// Set cancel if timeout. + cancel = true; + return Status::CANCEL; + } + ready_count = consumers; + return Status::NORMAL; + } + + void QueryInfoWrapper::notifyReady() + { + { + std::lock_guard lock(mutex_consumer); + ready.assign(consumers, true); + } + cv_consumer.notify_all(); + } + + void QueryInfoWrapper::notifyFinish() + { + { + std::lock_guard lock(mutex_consumer); + finish = true; + } + cv_consumer.notify_all(); + } + + QueryInfoWrapper::Status QueryInfoWrapper::waitHeader() + { + std::unique_lock lock(mutex_consumer); + bool status = cv_consumer.wait_for(lock, wait_timeout_seconds,[this] { return header == true || cancel; }); + if (!status) + return Status::TIMEOUT; + return Status::NORMAL; + } + + QueryInfoWrapper::Status QueryInfoWrapper::waitReadyOrFinish(int index) + { + std::unique_lock lock(mutex_consumer); + bool status = cv_consumer.wait_for( + lock, wait_timeout_seconds, [this, index] { return ready[index] || finish || cancel; }); + if (!status) + return Status::TIMEOUT; + if (finish) + return Status::FINISH; + if (cancel) + return Status::CANCEL; + ready[index] = false; + return Status::NORMAL; + } + + void QueryInfoWrapper::notifyProduce() + { + int res = 0; + { + std::lock_guard lock(mutex_producer); + res = --ready_count; + } + /// Notify producer only when all blocks are consumed, producer will be waken up once. + if (res == 0) + cv_producer.notify_one(); + } + + using QueryInfoMap = InnerMap>; /// Handles a connection after a responder is started (i.e. after getting a new call). class Call @@ -716,20 +854,24 @@ namespace private: void run(); - void saveQueryInfo(); - void runPlanFragment(); - void receiveQuery(); void receiveTicket(); void executeQuery(); + void storeQueryInfoWrapper(); + void loadQueryInfoWrapper(); + void processInput(); void initializeBlockInputStream(const Block & header); void createExternalTables(); void generateOutput(); + void produceOutput(); + void consumeOutput(); void finishQuery(); + void finishQueryInfo(); + void finishPipeline(); void onException(const Exception & exception); void onFatalError(); void releaseQueryIDAndSessionID(); @@ -750,8 +892,6 @@ namespace void throwIfFailedToSendResult(); void sendException(const Exception & exception); - static std::unique_ptr query_info_map; - const CallType call_type; std::unique_ptr responder; IServer & iserver; @@ -778,6 +918,10 @@ namespace GRPCResult result; GRPCTicket ticket; + static std::unique_ptr query_info_map; + String query_info_key; + std::shared_ptr query_info_wrapper; + bool initial_query_info_read = false; bool initial_ticket_read = false; bool finalize = false; @@ -827,19 +971,11 @@ namespace void Call::start(const std::function & on_finish_call_callback) { - void (Call::*function_to_run)() = nullptr; - if (call_type == CALL_SEND_SIMPLE) - function_to_run = &Call::saveQueryInfo; - else if (call_type == CALL_FRAGMENT_WITH_STREAM_OUTPUT) - function_to_run = &Call::runPlanFragment; - else - function_to_run = &Call::run; - - auto runner_function = [this, on_finish_call_callback, function_to_run] + auto runner_function = [this, on_finish_call_callback] { try { - (this->*function_to_run)(); + run(); } catch (...) { @@ -854,77 +990,39 @@ namespace { try { - setThreadName("GRPCServerCall"); - receiveQuery(); - executeQuery(); - processInput(); - generateOutput(); - finishQuery(); - } - catch (Exception & exception) - { - onException(exception); - } - catch (Poco::Exception & exception) - { - onException(Exception{Exception::CreateFromPocoTag{}, exception}); - } - catch (std::exception & exception) - { - onException(Exception{Exception::CreateFromSTDTag{}, exception}); - } - } - - void Call::saveQueryInfo() - { - try - { - setThreadName("GRPCServerCall"); - receiveQuery(); - auto query_id = query_info.query_id(); - auto stage_id = std::to_string(query_info.stage_id()); - auto sinks = query_info.sinks(); - /// TODO::Optimize flow - for(auto& node_id : sinks) + if (call_type == CALL_EXECUTE_PLAN_FRAGMENT) { - auto plan_fragment_id = query_id + "/" + stage_id + "/" + node_id; - LOG_DEBUG(log, "key: {}", plan_fragment_id); - auto status = query_info_map->insert(plan_fragment_id, std::move(query_info)); - if (!status) - { - throw Exception("Plan fragment id " + plan_fragment_id + " already exists", ErrorCodes::LOGICAL_ERROR); - } + setThreadName("GRPCServerExecutePlanFragment"); + + /// Include two steps: + /// 1.Store query info. + receiveQuery(); + storeQueryInfoWrapper(); + finishQueryInfo(); + + /// 2.Build and execute pipeline. + executeQuery(); /// Build pipeline. + produceOutput(); /// Execute pipeline. + finishPipeline(); + } + else if (call_type == CALL_FETCH_PLAN_FRAGMENT_RESULT) + { + setThreadName("GRPCServerFetchPlanFragmentResult"); + receiveTicket(); + loadQueryInfoWrapper(); + executeQuery(); + consumeOutput(); + finishQuery(); + } + else + { + setThreadName("GRPCServerCall"); + receiveQuery(); + executeQuery(); + processInput(); + generateOutput(); + finishQuery(); } - finishQuery(); - } - catch (Exception & exception) - { - onException(exception); - } - catch (Poco::Exception & exception) - { - onException(Exception{Exception::CreateFromPocoTag{}, exception}); - } - catch (std::exception & exception) - { - onException(Exception{Exception::CreateFromSTDTag{}, exception}); - } - } - - void Call::runPlanFragment() - { - try - { - setThreadName("GRPCServerCall"); - receiveTicket(); - auto plan_fragment_id = ticket.query_id() + "/" + std::to_string(ticket.stage_id()) + "/" + ticket.node_id(); - auto status = query_info_map->getAndErase(plan_fragment_id, query_info); - if (!status) - throw Exception("Plan fragment id " + plan_fragment_id + " not exists", ErrorCodes::LOGICAL_ERROR); - executeQuery(); - processInput(); - generateOutput(); - finishQuery(); } catch (Exception & exception) { @@ -958,11 +1056,16 @@ namespace readTicket(); - LOG_DEBUG(log, "Received ticket(stream name): {}", ticket.query_id() + "/" + std::to_string(ticket.stage_id()) + "/" + ticket.node_id()); + LOG_DEBUG(log, "Received ticket: {}", ticket.query_id() + "/" + std::to_string(ticket.stage_id()) + "/" + ticket.node_id()); } void Call::executeQuery() { + /// If this is from fetchPlanFragmentResult(), restore query_info from wrapper, + /// and initialize query_context, but don't build pipeline. + if (call_type == CALL_FETCH_PLAN_FRAGMENT_RESULT) + query_info = *query_info_wrapper->query_info; + /// Retrieve user credentials. std::string user = query_info.user_name(); std::string password = query_info.password(); @@ -990,28 +1093,27 @@ namespace query_context = session->makeQueryContext(); - auto genQueryPlanFragmentInfo = [](ContextMutablePtr & context_, GRPCQueryInfo & info_) + /// Set query plan fragment info + if (call_type == CALL_EXECUTE_PLAN_FRAGMENT) { std::vector> sources, sinks; - for (auto & source : info_.sources()) + for (const auto & source : query_info.sources()) { sources.emplace_back(std::make_shared(source)); } - for (auto & sink : info_.sinks()) + for (const auto & sink : query_info.sinks()) { sinks.emplace_back(std::make_shared(sink)); } Context::QueryPlanFragmentInfo fragmentInfo{ - .query_id = info_.query_id(), - .stage_id = info_.stage_id(), - .parent_stage_id = info_.parent_stage_id(), - .node_id = info_.node_id(), + .query_id = query_info.query_id(), + .stage_id = query_info.stage_id(), + .parent_stage_id = query_info.parent_stage_id(), + .node_id = query_info.node_id(), .sources = sources, .sinks = sinks }; - context_->setQueryPlanFragmentInfo(std::move(fragmentInfo)); - }; - - genQueryPlanFragmentInfo(query_context, query_info); + query_context->setQueryPlanFragmentInfo(std::move(fragmentInfo)); + } /// Prepare settings. SettingsChanges settings_changes; @@ -1049,6 +1151,13 @@ namespace interactive_delay = settings.interactive_delay; query_context->setProgressCallback([this](const Progress & value) { return progress.incrementPiecewiseAtomically(value); }); + if (call_type == CALL_FETCH_PLAN_FRAGMENT_RESULT) + { + query_context->setDefaultFormat("Native"); + output_format = "Native"; + return; + } + /// Parse the query. query_text = std::move(*(query_info.mutable_query())); const char * begin = query_text.data(); @@ -1115,6 +1224,27 @@ namespace io = ::DB::executeQuery(true, query, query_context); } + void Call::storeQueryInfoWrapper() + { + query_info_key = query_info.query_id() + "/" + toString(query_info.stage_id()); + auto res = query_info_map->insert(query_info_key, std::make_shared(&query_info, query_info.sinks_size())); + if (!res.second) + { + throw Exception("Query info key " + query_info_key + " already exists", ErrorCodes::LOGICAL_ERROR); + } + query_info_wrapper = res.first->second; + } + + void Call::loadQueryInfoWrapper() + { + query_info_key = ticket.query_id() + "/" + std::to_string(ticket.stage_id()); + auto res = query_info_map->get(query_info_key); + if (!res.second) + throw Exception("Query info key " + query_info_key + " not exists", ErrorCodes::LOGICAL_ERROR); + //query_info = *(res.first->query_info); + query_info_wrapper = res.first; + } + void Call::processInput() { if (!io.pipeline.pushing()) @@ -1433,6 +1563,136 @@ namespace output_format_processor->doWriteSuffix(); } + void Call::produceOutput() + { + if (!io.pipeline.initialized() || io.pipeline.pushing()) + return; + + if (io.pipeline.pulling()) + { + query_info_wrapper->setWaitTimeoutSeconds(query_context->getSettings().max_execution_time.totalSeconds()); + query_info_wrapper->notifyHeader(io.pipeline.getHeader()); + + /// Pull block from pipeline. + auto executor = std::make_shared(io.pipeline); + auto check_for_cancel = [this, &executor] { + if (query_info_wrapper->cancel) + { + result.set_cancelled(true); + cancelled = true; + LOG_DEBUG(log, "{} producer cancel pipeline executor.", query_info_key); + executor->cancel(); + return false; + } + return true; + }; + + Block block; + while (check_for_cancel()) + { + if (!executor->pull(block, interactive_delay / 1000)) + break; + + if (!check_for_cancel()) + break; + + if (block && !io.null_format) + { + block = materializeBlock(block); + query_info_wrapper->waitConsume(); + if (!check_for_cancel()) + break; + query_info_wrapper->blocks.assign(query_info_wrapper->consumers, block); + LOG_DEBUG(log, "{} produce {} block(s): {} rows, {} columns, {} bytes.", query_info_key, query_info_wrapper->consumers, block.rows(), block.columns(), block.bytes()); + query_info_wrapper->notifyReady(); + } + } + + /// Wait the last produced blocks to be consumed. + if (!query_info_wrapper->cancel) + query_info_wrapper->waitConsume(); + + if (!query_info_wrapper->cancel) + { + query_info_wrapper->totals = executor->getTotalsBlock(); + query_info_wrapper->extremes = executor->getExtremesBlock(); + query_info_wrapper->profile_info = executor->getProfileInfo(); + } + query_info_wrapper->notifyFinish(); + /// Wait all consumers to finish. + query_info_wrapper->waitConsume(); + LOG_DEBUG(log, "{} producer is {}.", query_info_key, (query_info_wrapper->cancel ? "cancelled" : "done")); + } + } + + void Call::consumeOutput() + { + int index = 0; + for (; index < query_info_wrapper->query_info->sinks().size(); ++index) + { + if (query_info_wrapper->query_info->sinks(index) == ticket.node_id()) + break; + } + + while (query_info_wrapper->waitHeader() == QueryInfoWrapper::Status::TIMEOUT); + + write_buffer.emplace(*result.mutable_output()); + output_format_processor = query_context->getOutputFormat(output_format, *write_buffer, query_info_wrapper->header); + output_format_processor->doWritePrefix(); + Stopwatch after_send_progress; + + /// Unless the input() function is used we are not going to receive input data anymore. + if (!input_function_is_used) + check_query_info_contains_cancel_only = true; + + Block block; + while (!query_info_wrapper->cancel) + { + auto status = query_info_wrapper->waitReadyOrFinish(index); + if (status == QueryInfoWrapper::Status::TIMEOUT) + continue; + if (query_info_wrapper->finish || query_info_wrapper->cancel) + break; + + block = query_info_wrapper->blocks[index]; + LOG_DEBUG(log, "{}/{} consume 1 block: {} rows, {} columns, {} bytes.", query_info_key, ticket.node_id(), block.rows(), block.columns(), block.bytes()); + query_info_wrapper->notifyProduce(); + + throwIfFailedToSendResult(); + if (query_info_wrapper->cancel) + break; + + if (block && !io.null_format) + output_format_processor->write(block); + + if (after_send_progress.elapsedMicroseconds() >= interactive_delay) + { + addProgressToResult(); + after_send_progress.restart(); + } + + addLogsToResult(); + + bool has_output = write_buffer->offset(); + if (has_output || result.has_progress() || result.logs_size()) + sendResult(); + + throwIfFailedToSendResult(); + } + + if (!query_info_wrapper->cancel) + { + addTotalsToResult(query_info_wrapper->totals); + addExtremesToResult(query_info_wrapper->extremes); + addProfileInfoToResult(query_info_wrapper->profile_info); + } + + output_format_processor->doWriteSuffix(); + /// Notify producer that current consumer is finished. + query_info_wrapper->notifyProduce(); + LOG_DEBUG(log, "{}/{} consumer is {}.", query_info_key, ticket.node_id(), (query_info_wrapper->cancel ? "cancelled" : "done")); + } + void Call::finishQuery() { finalize = true; @@ -1456,6 +1716,35 @@ namespace static_cast(waited_for_client_writing) / 1000000000ULL); } + void Call::finishQueryInfo() + { + finalize = true; + addProgressToResult(); + if (query_scope.has_value()) + { + query_scope->logPeakMemoryUsage(); + } + addLogsToResult(); + releaseQueryIDAndSessionID(); + sendResult(); + LOG_INFO( + log, + "Finished receiving query info in {} secs. (including reading by client: {}, writing by client: {})", + query_time.elapsedSeconds(), + static_cast(waited_for_client_reading) / 1000000000ULL, + static_cast(waited_for_client_writing) / 1000000000ULL); + } + + void Call::finishPipeline() + { + io.onFinish(); + close(); + LOG_INFO( + log, + "Finished executing pipeline of plan fragment {} in {} secs.", + query_info_key, query_time.elapsedSeconds()); + } + void Call::onException(const Exception & exception) { io.onException(); @@ -1528,6 +1817,8 @@ namespace query_scope.reset(); query_context.reset(); session.reset(); + if (call_type == CALL_EXECUTE_PLAN_FRAGMENT && !query_info_key.empty()) + query_info_map->erase(query_info_key); } void Call::readQueryInfo() diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index e76699608b9d..c9cc83c14cb8 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -190,6 +190,6 @@ service ClickHouse { rpc ExecuteQueryWithStreamOutput(QueryInfo) returns (stream Result) {} rpc ExecuteQueryWithStreamIO(stream QueryInfo) returns (stream Result) {} - rpc SendDistributedPlanParams(QueryInfo) returns (Result) {} - rpc ExecuteQueryFragmentWithStreamOutput(Ticket) returns (stream Result) {} + rpc ExecutePlanFragment(QueryInfo) returns (Result) {} + rpc FetchPlanFragmentResult(Ticket) returns (stream Result) {} } From 147cc1e08dd7eb017a8b4362e31d100282f23bf4 Mon Sep 17 00:00:00 2001 From: caspian Date: Fri, 17 Dec 2021 03:19:10 +0000 Subject: [PATCH 06/84] add cancel request support on new distributed plan --- src/Client/GRPCClient.cpp | 29 ++++++++- src/Client/GRPCClient.h | 6 +- src/Common/ErrorCodes.cpp | 1 + .../DistributedSourceExecutor.cpp | 4 ++ src/Server/GRPCServer.cpp | 62 ++++++++++++++++++- src/Server/grpc_protos/clickhouse_grpc.proto | 1 + 6 files changed, 100 insertions(+), 3 deletions(-) diff --git a/src/Client/GRPCClient.cpp b/src/Client/GRPCClient.cpp index efd743dac280..6aa44356f159 100644 --- a/src/Client/GRPCClient.cpp +++ b/src/Client/GRPCClient.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes { extern const int INVALID_GRPC_QUERY_INFO; extern const int GRPC_READ_ERROR; + extern const int GRPC_CANCEL_ERROR; } GRPCClient::GRPCClient(const String & addr_) @@ -53,8 +54,9 @@ GRPCResult GRPCClient::executePlanFragment(GRPCQueryInfo & query_info) } } -void GRPCClient::prepareRead(const GRPCTicket & ticket) +void GRPCClient::prepareRead(const GRPCTicket & ticket_) { + ticket = ticket_; auto ch = grpc::CreateChannel(addr, grpc::InsecureChannelCredentials()); std::shared_ptr ctx = std::make_shared(); auto stub = clickhouse::grpc::ClickHouse::NewStub(ch); @@ -88,4 +90,29 @@ Block GRPCClient::read() throw Exception("Read from grpc server " + addr + "failed, " + toString(result.exception().code()) + ", " + result.exception().display_text(), ErrorCodes::GRPC_READ_ERROR, true); } + +void GRPCClient::cancel() +{ + grpc::ClientContext ctx; + GRPCResult result; + + auto status = inner_context->stub->CancelPlanFragment(&ctx, ticket, &result); + + auto plan_fragment_id = ticket.query_id() + toString(ticket.stage_id()) + ticket.node_id(); + if (status.ok()) + { + if (result.cancelled()) + LOG_INFO(log, "Cancel success from node: {}, plan fragment id: {}", addr, plan_fragment_id); + else + { + throw Exception("Cancel failed from node: " + addr + ", plan fragment id: " + plan_fragment_id + ", code: " + toString(result.exception().code()) + ", " + result.exception().display_text(), ErrorCodes::GRPC_CANCEL_ERROR, true); + } + } + else + { + LOG_ERROR( + log, "Cancel failed from node {}, code: {}, plan fragment id: {}.", addr, status.error_code(), plan_fragment_id); + throw Exception(status.error_message() + ", " + result.exception().display_text(), ErrorCodes::GRPC_CANCEL_ERROR, true); + } +} } diff --git a/src/Client/GRPCClient.h b/src/Client/GRPCClient.h index 7b6b4fd624de..c8ca9b8e093f 100644 --- a/src/Client/GRPCClient.h +++ b/src/Client/GRPCClient.h @@ -33,12 +33,15 @@ class GRPCClient GRPCResult executePlanFragment(GRPCQueryInfo & g_query_info); /// Initialize reader and inner context. - void prepareRead(const GRPCTicket & ticket); + void prepareRead(const GRPCTicket & ticket_); /// Try to read a block from remote. /// If got EOF, an empty block will be returned, you can use if (!block) to check it. Block read(); + /// Cancel plan fragment (ticket associated with the prepareRead) + void cancel(); + private: struct InnerContext { @@ -61,6 +64,7 @@ class GRPCClient Poco::Logger * log; String addr; std::unique_ptr inner_context; + GRPCTicket ticket; }; } //#endif diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b1002e49cf92..b2e62d914acb 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -594,6 +594,7 @@ M(624, BAD_FILE_TYPE) \ M(625, IO_SETUP_ERROR) \ M(702, GRPC_READ_ERROR) \ + M(703, GRPC_CANCEL_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/QueryPipeline/DistributedSourceExecutor.cpp b/src/QueryPipeline/DistributedSourceExecutor.cpp index 8ce34c0b84e2..72146cca36c8 100644 --- a/src/QueryPipeline/DistributedSourceExecutor.cpp +++ b/src/QueryPipeline/DistributedSourceExecutor.cpp @@ -37,6 +37,9 @@ Block DistributedSourceExecutor::read() { auto block = client.read(); LOG_DEBUG(log, "Read block, rows: {}, columns: {}.", block.rows(), block.columns()); + if (!block) + finished = true; + return block; } catch (...) @@ -63,6 +66,7 @@ void DistributedSourceExecutor::cancel() return; LOG_DEBUG(log, "Cancel reading from {}.", *source); + client.cancel(); tryCancel("Cancelling query"); } diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index cd6cb5659370..421a9801de85 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -44,6 +44,7 @@ #include #include +#include using GRPCService = clickhouse::grpc::ClickHouse::AsyncService; using GRPCQueryInfo = clickhouse::grpc::QueryInfo; @@ -333,7 +334,8 @@ namespace CALL_WITH_STREAM_OUTPUT, /// ExecuteQueryWithStreamOutput() call CALL_WITH_STREAM_IO, /// ExecuteQueryWithStreamIO() call CALL_EXECUTE_PLAN_FRAGMENT, /// ExecutePlanFragment() call - CALL_FETCH_PLAN_FRAGMENT_RESULT, /// FetchPlanFragmentResult() call + CALL_FETCH_PLAN_FRAGMENT_RESULT, /// FetchPlanFragmentResult() call + CALL_CANCEL_PLAN_FRAGMENT, /// CancelPlanFragment() call CALL_MAX, }; @@ -347,6 +349,7 @@ namespace case CALL_WITH_STREAM_IO: return "ExecuteQueryWithStreamIO()"; case CALL_EXECUTE_PLAN_FRAGMENT: return "ExecutePlanFragment()"; case CALL_FETCH_PLAN_FRAGMENT_RESULT: return "FetchPlanFragmentResult()"; + case CALL_CANCEL_PLAN_FRAGMENT: return "CancelPlanFragment"; case CALL_MAX: break; } __builtin_unreachable(); @@ -601,6 +604,47 @@ namespace std::optional ticket; }; + template<> + class Responder : public BaseResponder + { + public: + void start(GRPCService & grpc_service, + grpc::ServerCompletionQueue & new_call_queue, + grpc::ServerCompletionQueue & notification_queue, + const CompletionCallback & callback) override + { + grpc_service.RequestCancelPlanFragment(&grpc_context, &ticket.emplace(), &writer, &new_call_queue, ¬ification_queue, getCallbackPtr(callback)); + } + + void read(GRPCQueryInfo &, const CompletionCallback &) override + { + throw Exception("Responder::read() should not be called", ErrorCodes::LOGICAL_ERROR); + } + + void read(GRPCTicket & ticket_, const CompletionCallback & callback) override + { + if (!ticket.has_value()) + callback(false); + ticket_ = std::move(ticket).value(); + ticket.reset(); + callback(true); + } + + void write(const GRPCResult &, const CompletionCallback &) override + { + throw Exception("Responder::write() should not be called", ErrorCodes::LOGICAL_ERROR); + } + + void writeAndFinish(const GRPCResult & result, const grpc::Status & status, const CompletionCallback & callback) override + { + writer.Finish(result, status, getCallbackPtr(callback)); + } + + private: + grpc::ServerAsyncResponseWriter writer{&grpc_context}; + std::optional ticket; + }; + std::unique_ptr makeResponder(CallType call_type) { switch (call_type) @@ -611,6 +655,7 @@ namespace case CALL_WITH_STREAM_IO: return std::make_unique>(); case CALL_EXECUTE_PLAN_FRAGMENT: return std::make_unique>(); case CALL_FETCH_PLAN_FRAGMENT_RESULT: return std::make_unique>(); + case CALL_CANCEL_PLAN_FRAGMENT: return std::make_unique>(); case CALL_MAX: break; } __builtin_unreachable(); @@ -1014,6 +1059,14 @@ namespace consumeOutput(); finishQuery(); } + else if (call_type == CALL_CANCEL_PLAN_FRAGMENT) + { + setThreadName("GRPCServerCancelPlanFragment"); + receiveTicket(); + loadQueryInfoWrapper(); + executeQuery(); + finishQuery(); + } else { setThreadName("GRPCServerCall"); @@ -1065,6 +1118,13 @@ namespace /// and initialize query_context, but don't build pipeline. if (call_type == CALL_FETCH_PLAN_FRAGMENT_RESULT) query_info = *query_info_wrapper->query_info; + else if (call_type == CALL_CANCEL_PLAN_FRAGMENT) + { + /// Find the old executor associated with the ticket, and cancel it + query_info_wrapper->cancel = true; + result.set_cancelled(true); + return; + } /// Retrieve user credentials. std::string user = query_info.user_name(); diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index c9cc83c14cb8..4474c6646ba2 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -192,4 +192,5 @@ service ClickHouse { rpc ExecutePlanFragment(QueryInfo) returns (Result) {} rpc FetchPlanFragmentResult(Ticket) returns (stream Result) {} + rpc CancelPlanFragment(Ticket) returns (Result) {} } From 9bb71d72c72f2c207b754c15d9cb05402625bb2c Mon Sep 17 00:00:00 2001 From: caspian Date: Thu, 16 Dec 2021 20:41:01 +0800 Subject: [PATCH 07/84] add kill on all support --- src/Interpreters/InterpreterKillQueryQuery.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 2b949266c170..585e288d66d5 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -195,7 +195,13 @@ class SyncKillQuerySource : public SourceWithProgress BlockIO InterpreterKillQueryQuery::execute() { - const auto & query = query_ptr->as(); + auto & query = query_ptr->as(); + + if (query.is_initial) + { + query.cluster = CLUSTER_TYPE_ALL; + return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccessForDDLOnCluster()); + } if (!query.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccessForDDLOnCluster()); From 5e8773e4e42b7a1cd4fdfbefd8dd79d61f57cf3f Mon Sep 17 00:00:00 2001 From: madianjun Date: Tue, 14 Dec 2021 19:09:28 +0800 Subject: [PATCH 08/84] Add distributed sort and limit. --- src/Client/GRPCClient.cpp | 22 +- src/Client/GRPCClient.h | 2 +- src/Interpreters/Context.cpp | 28 ++ src/Interpreters/Context.h | 3 +- src/Processors/QueryPlan/LimitStep.h | 11 + src/Processors/QueryPlan/QueryPlan.cpp | 332 ++++++++++++------ src/Processors/QueryPlan/QueryPlan.h | 22 +- src/Processors/QueryPlan/SortingStep.h | 3 + .../DistributedSourceExecutor.cpp | 2 +- src/Server/GRPCServer.cpp | 11 +- src/Server/grpc_protos/clickhouse_grpc.proto | 19 +- 11 files changed, 317 insertions(+), 138 deletions(-) diff --git a/src/Client/GRPCClient.cpp b/src/Client/GRPCClient.cpp index 6aa44356f159..f7ed9884aae0 100644 --- a/src/Client/GRPCClient.cpp +++ b/src/Client/GRPCClient.cpp @@ -34,14 +34,12 @@ GRPCClient::GRPCClient(const String & addr_) log = &Poco::Logger::get("GRPCClient(" + addr + ")"); } -GRPCResult GRPCClient::executePlanFragment(GRPCQueryInfo & query_info) +GRPCResult GRPCClient::executePlanFragment(const GRPCQueryInfo & query_info) { auto ch = grpc::CreateChannel(addr, grpc::InsecureChannelCredentials()); auto stub = clickhouse::grpc::ClickHouse::NewStub(ch); grpc::ClientContext ctx; GRPCResult result; - /// Set to native format, cause we decode result by NativeReader in the read function - query_info.set_output_format("Native"); grpc::Status status = stub->ExecutePlanFragment(&ctx, query_info, &result); if (status.ok()) @@ -49,7 +47,11 @@ GRPCResult GRPCClient::executePlanFragment(GRPCQueryInfo & query_info) else { LOG_ERROR( - log, "Send query info to {} failed, code: {}, plan fragment id: {}.", addr, status.error_code(), query_info.query_id() + toString(query_info.stage_id()) + query_info.node_id()); + log, + "Send query info to {} failed, code: {}, plan fragment id: {}.", + addr, + status.error_code(), + query_info.initial_query_id() + toString(query_info.stage_id()) + query_info.node_id()); throw Exception(status.error_message() + ", " + result.exception().display_text(), ErrorCodes::INVALID_GRPC_QUERY_INFO, true); } } @@ -74,7 +76,12 @@ Block GRPCClient::read() { if (result.exception().code() != 0) { - LOG_ERROR(log, "Read from {} failed, exception.code: {}, exception.text: {}.", addr, result.exception().code(), result.exception().display_text()); + LOG_ERROR( + log, + "Read from {} failed, exception.code: {}, exception.text: {}.", + addr, + result.exception().code(), + result.exception().display_text()); throw Exception(result.exception().display_text(), ErrorCodes::INVALID_GRPC_QUERY_INFO, true); } @@ -88,7 +95,10 @@ Block GRPCClient::read() return block; } - throw Exception("Read from grpc server " + addr + "failed, " + toString(result.exception().code()) + ", " + result.exception().display_text(), ErrorCodes::GRPC_READ_ERROR, true); + throw Exception( + "Read from grpc server " + addr + "failed, " + toString(result.exception().code()) + ", " + result.exception().display_text(), + ErrorCodes::GRPC_READ_ERROR, + true); } void GRPCClient::cancel() diff --git a/src/Client/GRPCClient.h b/src/Client/GRPCClient.h index c8ca9b8e093f..0d4bcd4aff02 100644 --- a/src/Client/GRPCClient.h +++ b/src/Client/GRPCClient.h @@ -30,7 +30,7 @@ class GRPCClient ~GRPCClient() = default; /// Send params of plan fragment to remote, and execute it. - GRPCResult executePlanFragment(GRPCQueryInfo & g_query_info); + GRPCResult executePlanFragment(const GRPCQueryInfo & query_info); /// Initialize reader and inner context. void prepareRead(const GRPCTicket & ticket_); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 756b0c74fe2e..83860244a343 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1260,6 +1260,34 @@ void Context::setCurrentQueryId(const String & query_id) client_info.initial_query_id = client_info.current_query_id; } +String Context::generateQueryId() const +{ + /// Generate random UUID, but using lower quality RNG, + /// because Poco::UUIDGenerator::generateRandom method is using /dev/random, that is very expensive. + /// NOTE: Actually we don't need to use UUIDs for query identifiers. + /// We could use any suitable string instead. + union + { + char bytes[16]; + struct + { + UInt64 a; + UInt64 b; + } words; + UUID uuid{}; + } random; + + random.words.a = thread_local_rng(); //-V656 + random.words.b = thread_local_rng(); //-V656 + + /// Use protected constructor. + struct QueryUUID : Poco::UUID + { + QueryUUID(const char * bytes, Poco::UUID::Version version) : Poco::UUID(bytes, version) { } + }; + return QueryUUID(random.bytes, Poco::UUID::UUID_RANDOM).toString(); +} + void Context::killCurrentQuery() { if (process_list_elem) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 2f19e387400e..e96d5be100b0 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -298,7 +298,7 @@ class Context: public std::enable_shared_from_this public: struct QueryPlanFragmentInfo { - String query_id; + String initial_query_id; int stage_id; /// Stage that should be executed on this replica. int parent_stage_id; String node_id; /// This is myself replica name and grpc port. @@ -538,6 +538,7 @@ class Context: public std::enable_shared_from_this /// exists because it should be set before databases loading. void setCurrentDatabaseNameInGlobalContext(const String & name); void setCurrentQueryId(const String & query_id); + String generateQueryId() const; void killCurrentQuery(); diff --git a/src/Processors/QueryPlan/LimitStep.h b/src/Processors/QueryPlan/LimitStep.h index f5bceeb29c7a..b1e0ec095db6 100644 --- a/src/Processors/QueryPlan/LimitStep.h +++ b/src/Processors/QueryPlan/LimitStep.h @@ -16,6 +16,17 @@ class LimitStep : public ITransformingStep bool with_ties_ = false, /// Limit with ties. SortDescription description_ = {}); + LimitStep(const DataStream & input_stream_, const LimitStep & limit_step) + : LimitStep( + input_stream_, + limit_step.limit, + limit_step.offset, + limit_step.always_read_till_end, + limit_step.with_ties, + limit_step.description) + { + } + String getName() const override { return "Limit"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index fd2ec1b01da5..1eaa2800f70f 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -10,6 +10,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -88,7 +91,10 @@ void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vectorchildren.emplace_back(plan->root); + plan->root->parent = root; + } for (auto & plan : plans) { @@ -130,6 +136,7 @@ void QueryPlan::addStep(QueryPlanStepPtr step) "step header: " + step_header.dumpStructure(), ErrorCodes::LOGICAL_ERROR); nodes.emplace_back(Node{.step = std::move(step), .children = {root}}); + root->parent = &nodes.back(); root = &nodes.back(); return; } @@ -145,6 +152,18 @@ void QueryPlan::reset() nodes.clear(); } +void QueryPlan::checkShuffle(Node * current_node, Node * child_node, CheckShuffleResult & result) +{ + result.child_sorting_step = dynamic_cast(child_node->step.get()); + if (result.child_sorting_step) + result.current_limit_step = dynamic_cast(current_node->step.get()); + else + result.child_limit_step = dynamic_cast(child_node->step.get()); + + if ((result.child_sorting_step && !result.current_limit_step) || result.child_limit_step) + result.is_shuffle = true; +} + void QueryPlan::buildStages(ContextPtr) { LOG_DEBUG(log, "Build stages."); @@ -173,7 +192,7 @@ void QueryPlan::buildStages(ContextPtr) /// Used for creating stage. int stage_id = -1; - Node * stage_root_node = nullptr; + Node * child_node = nullptr; Stage * last_stage = nullptr; while (!stack.empty()) @@ -185,19 +204,21 @@ void QueryPlan::buildStages(ContextPtr) ++frame.visited_children; one_child_is_visited = false; - /// TODO: This is shuffle, construct a new stage -// if (false) -// { -// stage_id++; -// last_stage = createStage(stage_id, last_stage, stage_root_node); -// } + /// This is shuffle, create a new stage + CheckShuffleResult result; + checkShuffle(frame.node, child_node, result); + if (result.is_shuffle) + { + ++stage_id; + last_stage = createStage(stage_id, last_stage, child_node); + } } size_t next_child = frame.visited_children; if (next_child == frame.node->children.size()) { - LOG_DEBUG(log, "Visited step: {}", frame.node->step->getName()); - stage_root_node = frame.node; + LOG_DEBUG(log, "Visit step: {}", frame.node->step->getName()); + child_node = frame.node; one_child_is_visited = true; stack.pop(); } @@ -207,7 +228,7 @@ void QueryPlan::buildStages(ContextPtr) /// At last, append a shuffle for converging data. ++stage_id; - last_stage = createStage(stage_id, last_stage, stage_root_node); + last_stage = createStage(stage_id, last_stage, child_node); /// Create result stage. ++stage_id; @@ -216,9 +237,15 @@ void QueryPlan::buildStages(ContextPtr) void QueryPlan::scheduleStages(ContextPtr context) { + /// Use initial query id to build the plan fragment id. + const String & initial_query_id = context->getClientInfo().current_query_id; + + /// Get my replica grpc address + String my_replica = context->getMacros()->getValue("replica") + ":" + toString(context->getServerPort("grpc_port")); + /// Retrieve all replicas. std::unordered_map replicas = context->getClustersWatcher().getContainer(); - LOG_DEBUG(log, "Schedule stages across {} replicas.", replicas.size()); + LOG_DEBUG(log, "Schedule stages for query id {} across {} replicas.", initial_query_id, replicas.size()); std::vector> store_replicas, compute_replicas; for (const auto & replica : replicas) { @@ -243,32 +270,59 @@ void QueryPlan::scheduleStages(ContextPtr context) } LOG_DEBUG(log, "{} store, {} compute.", store_replicas.size(), compute_replicas.size()); - auto fillStage = [&store_replicas, &compute_replicas](Stage * stage) + /// Fill sources. + auto fillSources = [](Stage * stage) { - if (stage->parents.empty()) /// Leaf stage. + int num_sources = 0; + for (Stage * parent : stage->parents) + { + num_sources += parent->workers.size(); + } + stage->sources.reserve(num_sources); + for (Stage * parent : stage->parents) { - /// Fill executors. - stage->executors.reserve(store_replicas.size()); - stage->executors.insert(stage->executors.end(), store_replicas.begin(), store_replicas.end()); + stage->sources.insert(stage->sources.end(), parent->workers.begin(), parent->workers.end()); + } + }; + + auto fillStage = [&store_replicas, &compute_replicas, this, &my_replica, fillSources](Stage * stage) + { + /// Leaf stage. + if (stage->parents.empty()) + { + /// Fill workers. + stage->workers.reserve(store_replicas.size()); + stage->workers.insert(stage->workers.end(), store_replicas.begin(), store_replicas.end()); /// Leaf stage's sources should be empty. + return; } - else /// Non-leaf stage. + + /// Result stage. + if (stage == result_stage) { - /// Fill executors. - stage->executors.reserve(compute_replicas.size()); - stage->executors.insert(stage->executors.end(), compute_replicas.begin(), compute_replicas.end()); - /// Fill sources. - int num_sources = 0; - for (Stage * parent : stage->parents) - { - num_sources += parent->executors.size(); - } - stage->sources.reserve(num_sources); - for (Stage * parent : stage->parents) + stage->workers.emplace_back(std::make_shared(my_replica)); + /// Optimize: the result stage has one single source and its parent is not leaf stage, + /// place its parent stage on the same worker as result stage. + if (stage->parents.size() == 1 && stage->sources.size() == 1 && !stage->parents[0]->sources.empty()) { - stage->sources.insert(stage->sources.end(), parent->executors.begin(), parent->executors.end()); + auto * parent = stage->parents.front(); + parent->workers.front() = stage->workers.front(); + stage->sources.front() = parent->workers.front(); + + /// Another solution: merge result stage into its parent stage. + //LOG_DEBUG(log, "Result stage {} moves forward to parent stage {}.", result_stage->id, result_stage->parents[0]->id); + //result_stage = result_stage->parents[0]; } + else + fillSources(stage); + + return; } + + /// Intermediate stage. + stage->workers.reserve(compute_replicas.size()); + stage->workers.insert(stage->workers.end(), compute_replicas.begin(), compute_replicas.end()); + fillSources(stage); }; struct Frame @@ -295,7 +349,7 @@ void QueryPlan::scheduleStages(ContextPtr context) size_t next_parent = frame.visited_parents; if (next_parent == frame.stage->parents.size()) { - LOG_DEBUG(log, "Visited stage: {}", frame.stage->id); + LOG_DEBUG(log, "Visit stage: {}", frame.stage->id); fillStage(frame.stage); @@ -309,7 +363,6 @@ void QueryPlan::scheduleStages(ContextPtr context) } /// Send plan fragment params. - const String & query_id = context->getClientInfo().initial_query_id; for (auto & stage : stages) { /// Don't send result stage. @@ -336,10 +389,8 @@ void QueryPlan::scheduleStages(ContextPtr context) header.columns()); } - /// Get my replica grpc address - String my_replica = context->getMacros()->getValue("replica") + ":" + toString(context->getServerPort("grpc_port")); auto distributed_source_step = std::make_unique( - header, parent_stage->executors, query_id, result_stage->id, parent_stage->id, my_replica, context); + header, parent_stage->workers, initial_query_id, result_stage->id, parent_stage->id, *result_stage->workers.front(), context); reset(); addStep(std::move(distributed_source_step)); { @@ -347,43 +398,50 @@ void QueryPlan::scheduleStages(ContextPtr context) LOG_DEBUG( log, "Local plan fragment:\n{}", - debugLocalPlanFragment(query_id, result_stage->id, my_replica, std::vector{root})); + debugLocalPlanFragment(initial_query_id, result_stage->id, *result_stage->workers.front(), std::vector{root})); } continue; } /// Fill sinks. - if (!stage.child->executors.empty()) + if (!stage.child->workers.empty()) { - stage.sinks.reserve(stage.child->executors.size()); - stage.sinks.insert(stage.sinks.end(), stage.child->executors.begin(), stage.child->executors.end()); + stage.sinks.reserve(stage.child->workers.size()); + stage.sinks.insert(stage.sinks.end(), stage.child->workers.begin(), stage.child->workers.end()); } - LOG_DEBUG(log, "Stage {} has {} executors.", stage.id, stage.executors.size()); - /// Send to each remote executor. - for (const auto & executor : stage.executors) + LOG_DEBUG(log, "Stage {} has {} workers.", stage.id, stage.workers.size()); + + /// Create query info. + GRPCQueryInfo query_info; + query_info.set_output_format("Native"); + if (!context->getSelectQuery().empty()) + query_info.set_query(context->getSelectQuery()); /// For "insert into ... select" + else + query_info.set_query(context->getClientInfo().initial_query); + query_info.set_query_id(context->generateQueryId()); + query_info.set_initial_query_id(initial_query_id); + query_info.set_stage_id(stage.id); + if (!stage.parents.empty()) + query_info.set_parent_stage_id(stage.parents[0]->id); + else + query_info.set_parent_stage_id(-1); + + for (const auto & source : stage.sources) { - GRPCQueryInfo query_info; - if (!context->getSelectQuery().empty()) - query_info.set_query(context->getSelectQuery()); - else - query_info.set_query(context->getClientInfo().initial_query); - query_info.set_query_id(query_id); - query_info.set_stage_id(stage.id); - if (!stage.parents.empty()) - query_info.set_parent_stage_id(stage.parents[0]->id); - query_info.set_node_id(*executor); - for (const auto & source : stage.sources) - { - query_info.add_sources(*source); - } - for (const auto & sink : stage.sinks) - { - query_info.add_sinks(*sink); - } - LOG_DEBUG(log, "Plan fragment to send:\nquery: {}\n{}", query_info.query(), debugRemotePlanFragment(*executor, query_id, &stage)); + query_info.add_sources(*source); + } + for (const auto & sink : stage.sinks) + { + query_info.add_sinks(*sink); + } + /// Send query info to each remote worker. + for (const auto & worker : stage.workers) + { + query_info.set_node_id(*worker); + LOG_DEBUG(log, "Remote plan fragment:\n{}", debugRemotePlanFragment(query_info.query(), *worker, initial_query_id, &stage)); - GRPCClient cli(*executor); + GRPCClient cli(*worker); auto result = cli.executePlanFragment(query_info); LOG_DEBUG(log, "GRPCClient got result, exception code: {}, exception text: {}.", result.exception().code(), result.exception().display_text()); } @@ -392,12 +450,16 @@ void QueryPlan::scheduleStages(ContextPtr context) void QueryPlan::buildPlanFragment(ContextPtr context) { - LOG_DEBUG(log, "Build plan fragment."); + const auto & query_distributed_plan_info = context->getQueryPlanFragmentInfo(); + int my_stage_id = query_distributed_plan_info.stage_id; + LOG_DEBUG( + log, + "Build plan fragment: stage {} parent stage {}.", + my_stage_id, + query_distributed_plan_info.parent_stage_id); /// Get my replica grpc address String my_replica = context->getMacros()->getValue("replica") + ":" + toString(context->getServerPort("grpc_port")); - const auto & query_distributed_plan_info = context->getQueryPlanFragmentInfo(); - int my_stage_id = query_distributed_plan_info.stage_id; struct Frame { @@ -421,39 +483,82 @@ void QueryPlan::buildPlanFragment(ContextPtr context) if (one_child_is_visited) { - /// TODO: This is a shuffle dependency between current node and the last visited child. -// if (false) -// { -// stage_id++; -// /// Add a DistributedSourceStep between current node and child node. -// if (stage_id == query_distributed_plan_info.parent_stage_id) /// TODO: if(query_distributed_plan_info.parent_id_to_sources.contains(stage_id)) -// { -// /// Create a DistributedSourceStep. -// const auto & header = child_node->step->getOutputStream().header; -// const String & query_id = context->getClientInfo().initial_query_id; -// const auto & sources = query_distributed_plan_info.sources; // TODO: query_distributed_plan_info.parent_id_to_sources[stage_id]; -// auto distributed_source_step = std::make_unique( -// header, sources, query_id, my_stage_id, stage_id, my_replica, context); -// -// /// Reuse child node, but replace its step with DistributedSourceStep. -// assert(child_node == frame.node->children[frame.visited_children]); -// child_node->step = std::move(distributed_source_step); -// child_node->children.clear(); -// distributed_source_nodes.emplace_back(child_node); -// } -// else if (stage_id == my_stage_id) -// { -// root = child_node; -// { -// /// Only for debug. -// LOG_DEBUG( -// log, -// "Local plan fragment:\n{}", -// debugLocalPlanFragment(query_distributed_plan_info.query_id, stage_id, my_replica, distributed_source_nodes)); -// } -// return; -// } -// } + CheckShuffleResult result; + checkShuffle(frame.node, child_node, result); + + /// This is a shuffle dependency between current node and the last visited child. + if (result.is_shuffle) + { + ++stage_id; + /// Add a DistributedSourceStep between current node and child node. + if (stage_id == query_distributed_plan_info.parent_stage_id) /// TODO: if(query_distributed_plan_info.parent_id_to_sources.contains(stage_id)) + { + assert(child_node == frame.node->children[frame.visited_children]); + + auto addStep = [this, &stage_id](QueryPlanStepPtr step, const String & description, Node * & node) + { + step->setStepDescription(description); + if (!node) + nodes.emplace_back(Node{.step = std::move(step)}); + else + { + nodes.emplace_back(Node{.step = std::move(step), .children = {node}}); + node->parent = &nodes.back(); + } + node = &nodes.back(); + LOG_DEBUG(log, "Add step: {}, parent stage: {}", step->getName(), stage_id); + }; + + /// Create DistributedSourceStep. + const auto & header = child_node->step->getOutputStream().header; + const auto & sources + = query_distributed_plan_info.sources; // TODO: query_distributed_plan_info.parent_id_to_sources[stage_id]; + auto distributed_source_step + = std::make_unique(header, sources, query_distributed_plan_info.initial_query_id, my_stage_id, stage_id, my_replica, context); + Node * new_node = nullptr; + addStep(std::move(distributed_source_step), "", new_node); + distributed_source_nodes.emplace_back(new_node); /// For debug + + /// If parent stage has order by, add SortingStep + if (result.child_sorting_step) + { + + auto merging_sorted = std::make_unique(new_node->step->getOutputStream(), *result.child_sorting_step); + addStep(std::move(merging_sorted), "Merge sorted streams for distributed ORDER BY", new_node); + } + + /// If parent stage has limit, add LimitStep + if (result.child_limit_step) + { + assert(child_node->children.size() == 1); + const SortingStep * grandchild_sorting_step = dynamic_cast(child_node->children[0]->step.get()); + if (grandchild_sorting_step) + { + auto merging_sorted + = std::make_unique(new_node->step->getOutputStream(), *grandchild_sorting_step); + addStep(std::move(merging_sorted), "Merge sorted streams for distributed ORDER BY", new_node); + } + + auto limit = std::make_unique(new_node->step->getOutputStream(), *result.child_limit_step); + addStep(std::move(limit), "distributed LIMIT", new_node); + } + + /// Add new child node to current node. + frame.node->children[frame.visited_children] = new_node; + } + else if (stage_id == my_stage_id) + { + root = child_node; + { + /// Only for debug. + LOG_DEBUG( + log, + "Local plan fragment:\n{}", + debugLocalPlanFragment(query_distributed_plan_info.initial_query_id, stage_id, my_replica, distributed_source_nodes)); + } + return; + } + } ++frame.visited_children; one_child_is_visited = false; @@ -462,7 +567,7 @@ void QueryPlan::buildPlanFragment(ContextPtr context) size_t next_child = frame.visited_children; if (next_child == frame.node->children.size()) { - LOG_DEBUG(log, "Visited step: {}", frame.node->step->getName()); + LOG_DEBUG(log, "Visit step: {}", frame.node->step->getName()); child_node = frame.node; one_child_is_visited = true; stack.pop(); @@ -471,7 +576,7 @@ void QueryPlan::buildPlanFragment(ContextPtr context) stack.push(Frame{.node = frame.node->children[next_child]}); } - /// Check the result stage. + /// Check the last stage(in fact it's the parent stage of the result stage). ++stage_id; if (stage_id == my_stage_id) { @@ -481,7 +586,7 @@ void QueryPlan::buildPlanFragment(ContextPtr context) LOG_DEBUG( log, "Local plan fragment:\n{}", - debugLocalPlanFragment(query_distributed_plan_info.query_id, stage_id, my_replica, distributed_source_nodes)); + debugLocalPlanFragment(query_distributed_plan_info.initial_query_id, stage_id, my_replica, distributed_source_nodes)); } return; } @@ -521,35 +626,42 @@ String QueryPlan::debugLocalPlanFragment(const String & query_id, int stage_id, { WriteBufferFromOwnString buf; ExplainPlanOptions options; - buf << "fragment id: " << query_id << "/" << stage_id << "/" << node_id << "\n"; - buf << "distributed source " << distributed_source_nodes.size() << " nodes:\n"; - for (const auto node : distributed_source_nodes) + buf << "------ Local Plan Fragment ------\n"; + buf << "Fragment ID: " << query_id << "/" << stage_id << "/" << node_id; + buf.write('\n'); + buf << "Distributed Source Nodes: " << distributed_source_nodes.size(); + buf.write('\n'); + for (size_t i = 0; i < distributed_source_nodes.size(); ++i) { + const Node * node = distributed_source_nodes[i]; auto distributed_source_step = dynamic_cast(node->step.get()); - buf << distributed_source_step->getName() << ", sources: "; + buf << "[" << i << "]" << distributed_source_step->getName() << ", sources: "; for (const auto & source : distributed_source_step->getSources()) buf << *source << " "; buf.write('\n'); } - buf << "plan fragment:\n"; + buf << "Plan Fragment:\n"; explainPlan(buf, options); return buf.str(); } -String QueryPlan::debugRemotePlanFragment(const String & receiver, const String & query_id, const Stage * stage) +String QueryPlan::debugRemotePlanFragment(const String & query, const String & receiver, const String & query_id, const Stage * stage) { WriteBufferFromOwnString buf; - buf << "receiver: " << receiver; + buf << "------ Remote Plan Fragment ------\n"; + buf << "Query: " << query; + buf.write('\n'); + buf << "Receiver: " << receiver; buf.write('\n'); - buf << "fragment id: " << query_id << "/" << stage->id << "/" << receiver; + buf << "Fragment ID: " << query_id << "/" << stage->id << "/" << receiver; buf.write('\n'); - buf << "sources: "; + buf << "Sources: "; for (const auto & source : stage->sources) { buf << *source << " "; } buf.write('\n'); - buf << "sinks: "; + buf << "Sinks: "; for (const auto & sink : stage->sinks) { buf << *sink << " "; diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 9250089676c6..44ead12e2378 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -28,6 +27,9 @@ using QueryPlanPtr = std::unique_ptr; class Pipe; +class SortingStep; +class LimitStep; + struct QueryPlanOptimizationSettings; struct BuildQueryPipelineSettings; @@ -58,6 +60,7 @@ class QueryPlan void optimize(const QueryPlanOptimizationSettings & optimization_settings); void reset(); + void buildStages(ContextPtr context); /// Used by initial node. void scheduleStages(ContextPtr context); /// Used by initial node. void buildPlanFragment(ContextPtr context); /// Used by non-initial nodes. @@ -107,19 +110,20 @@ class QueryPlan { QueryPlanStepPtr step; std::vector children = {}; + Node * parent = nullptr; }; using Nodes = std::list; struct Stage { - int id; + int id; /// Current stage id. std::vector parents = {}; /// Previous stages that current stage directly depends on. Stage * child = nullptr; - std::vector> executors; /// Replicas that current stage should be executed on. + std::vector> workers; /// Replicas that current stage should be executed on. std::vector> sources; /// Parents' executors. std::vector> sinks; /// Child's executors. - Node * node; /// Current stage's root node + Node * node; /// Current stage's root node. }; /// Note: do not use vector, otherwise pointers to elements in it will be invalidated when vector increases. @@ -136,8 +140,16 @@ class QueryPlan }; using PlanFragmentInfoPtr = std::shared_ptr; + struct CheckShuffleResult + { + bool is_shuffle = false; + SortingStep * child_sorting_step = nullptr; + LimitStep * current_limit_step = nullptr; + LimitStep * child_limit_step = nullptr; + }; + void checkShuffle(Node * current_node, Node * child_node, CheckShuffleResult & result); String debugLocalPlanFragment(const String & query_id, int stage_id, const String & node_id, const std::vector distributed_source_nodes); - String debugRemotePlanFragment(const String & receiver, const String & query_id, const Stage * stage); + String debugRemotePlanFragment(const String & query, const String & receiver, const String & query_id, const Stage * stage); private: Nodes nodes; diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index 8e253e71f441..b3897e202360 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -39,6 +39,9 @@ class SortingStep : public ITransformingStep size_t max_block_size_, UInt64 limit_ = 0); + SortingStep(const DataStream & input_stream_, const SortingStep & sorting_step) + : SortingStep(input_stream_, sorting_step.result_description, sorting_step.max_block_size, sorting_step.limit){} + String getName() const override { return "Sorting"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; diff --git a/src/QueryPipeline/DistributedSourceExecutor.cpp b/src/QueryPipeline/DistributedSourceExecutor.cpp index 72146cca36c8..aa5852f3e65a 100644 --- a/src/QueryPipeline/DistributedSourceExecutor.cpp +++ b/src/QueryPipeline/DistributedSourceExecutor.cpp @@ -18,7 +18,7 @@ DistributedSourceExecutor::DistributedSourceExecutor( , log(&Poco::Logger::get("DistributedSourceExecutor(" + query_id + "/" + toString(stage_id) + "/" + node_id + ")")) { GRPCTicket ticket; - ticket.set_query_id(query_id); + ticket.set_initial_query_id(query_id); ticket.set_stage_id(stage_id); ticket.set_node_id(node_id); client.prepareRead(ticket); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 421a9801de85..606fe4e9a3ce 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1109,7 +1109,7 @@ namespace readTicket(); - LOG_DEBUG(log, "Received ticket: {}", ticket.query_id() + "/" + std::to_string(ticket.stage_id()) + "/" + ticket.node_id()); + LOG_DEBUG(log, "Received ticket: {}", ticket.initial_query_id() + "/" + std::to_string(ticket.stage_id()) + "/" + ticket.node_id()); } void Call::executeQuery() @@ -1166,7 +1166,7 @@ namespace sinks.emplace_back(std::make_shared(sink)); } Context::QueryPlanFragmentInfo fragmentInfo{ - .query_id = query_info.query_id(), + .initial_query_id = query_info.initial_query_id(), .stage_id = query_info.stage_id(), .parent_stage_id = query_info.parent_stage_id(), .node_id = query_info.node_id(), @@ -1184,6 +1184,8 @@ namespace query_context->checkSettingsConstraints(settings_changes); query_context->applySettingsChanges(settings_changes); + query_context->getClientInfo().query_kind = ClientInfo::QueryKind(query_info.query_kind()); + query_context->getClientInfo().initial_query_id = query_info.initial_query_id(); query_context->setCurrentQueryId(query_info.query_id()); query_scope.emplace(query_context); @@ -1286,7 +1288,7 @@ namespace void Call::storeQueryInfoWrapper() { - query_info_key = query_info.query_id() + "/" + toString(query_info.stage_id()); + query_info_key = query_info.initial_query_id() + "/" + toString(query_info.stage_id()); auto res = query_info_map->insert(query_info_key, std::make_shared(&query_info, query_info.sinks_size())); if (!res.second) { @@ -1297,11 +1299,10 @@ namespace void Call::loadQueryInfoWrapper() { - query_info_key = ticket.query_id() + "/" + std::to_string(ticket.stage_id()); + query_info_key = ticket.initial_query_id() + "/" + std::to_string(ticket.stage_id()); auto res = query_info_map->get(query_info_key); if (!res.second) throw Exception("Query info key " + query_info_key + " not exists", ErrorCodes::LOGICAL_ERROR); - //query_info = *(res.first->query_info); query_info_wrapper = res.first; } diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index 4474c6646ba2..8270f0ab9991 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -63,7 +63,7 @@ message Compression { message Ticket { // same as QueryInfo.query_id - string query_id = 1; + string initial_query_id = 1; int32 stage_id = 2; string node_id = 3; } @@ -110,15 +110,16 @@ message QueryInfo { /// If not set the compression settings from the configuration file will be used. Compression result_compression = 17; - /// the stage id of target host which receives this request - int32 stage_id = 18; - /// the id of the target host - string node_id = 19; + uint32 query_kind = 18; + /// Below are used in distributed plan execution. + string initial_query_id = 19; + int32 stage_id = 20; + string node_id = 21; /// TODO use map to represent the relation between source and parent stage id - int32 parent_stage_id = 20; - repeated string sources = 21; - /// replica_name:grpc_port of sinks - repeated string sinks = 22; + int32 parent_stage_id = 22; + repeated string sources = 23; + /// Format is "replica_name:grpc_port" + repeated string sinks = 24; } enum LogsLevel { From 576de4a1c31c16e1802230bd3f73aad46272a1f4 Mon Sep 17 00:00:00 2001 From: madianjun Date: Wed, 15 Dec 2021 16:48:50 +0800 Subject: [PATCH 09/84] Change stage to use multiple parent stages. --- src/Interpreters/Context.h | 6 +- src/Processors/QueryPlan/QueryPlan.cpp | 128 +++++++++---------- src/Processors/QueryPlan/QueryPlan.h | 3 +- src/Server/GRPCServer.cpp | 13 +- src/Server/grpc_protos/clickhouse_grpc.proto | 9 +- 5 files changed, 77 insertions(+), 82 deletions(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index e96d5be100b0..2f748dc45d7a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -300,10 +300,10 @@ class Context: public std::enable_shared_from_this { String initial_query_id; int stage_id; /// Stage that should be executed on this replica. - int parent_stage_id; String node_id; /// This is myself replica name and grpc port. - std::vector> sources; /// Point to replicas that sending data. - std::vector> sinks; /// Point to replicas that receiving data. +// int parent_stage_id; + std::unordered_map>> parent_sources; /// Mapping of parent id and its workers. + std::vector> sinks; /// Point to workers that receiving data. }; // Top-level OpenTelemetry trace context for the query. Makes sense only for a query context. diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 1eaa2800f70f..ea6058996ff0 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -270,22 +270,7 @@ void QueryPlan::scheduleStages(ContextPtr context) } LOG_DEBUG(log, "{} store, {} compute.", store_replicas.size(), compute_replicas.size()); - /// Fill sources. - auto fillSources = [](Stage * stage) - { - int num_sources = 0; - for (Stage * parent : stage->parents) - { - num_sources += parent->workers.size(); - } - stage->sources.reserve(num_sources); - for (Stage * parent : stage->parents) - { - stage->sources.insert(stage->sources.end(), parent->workers.begin(), parent->workers.end()); - } - }; - - auto fillStage = [&store_replicas, &compute_replicas, this, &my_replica, fillSources](Stage * stage) + auto fillStage = [&store_replicas, &compute_replicas, this, &my_replica](Stage * stage) { /// Leaf stage. if (stage->parents.empty()) @@ -301,28 +286,25 @@ void QueryPlan::scheduleStages(ContextPtr context) if (stage == result_stage) { stage->workers.emplace_back(std::make_shared(my_replica)); - /// Optimize: the result stage has one single source and its parent is not leaf stage, + /// Optimize: + /// the result stage has only one parent worker and its parent stage is not leaf stage, /// place its parent stage on the same worker as result stage. - if (stage->parents.size() == 1 && stage->sources.size() == 1 && !stage->parents[0]->sources.empty()) + if (stage->parents.size() == 1) { auto * parent = stage->parents.front(); - parent->workers.front() = stage->workers.front(); - stage->sources.front() = parent->workers.front(); + if (parent->workers.size() == 1 && !parent->parents.empty()) + parent->workers.front() = stage->workers.front(); /// Another solution: merge result stage into its parent stage. //LOG_DEBUG(log, "Result stage {} moves forward to parent stage {}.", result_stage->id, result_stage->parents[0]->id); //result_stage = result_stage->parents[0]; } - else - fillSources(stage); - return; } /// Intermediate stage. stage->workers.reserve(compute_replicas.size()); stage->workers.insert(stage->workers.end(), compute_replicas.begin(), compute_replicas.end()); - fillSources(stage); }; struct Frame @@ -369,36 +351,44 @@ void QueryPlan::scheduleStages(ContextPtr context) if (&stage == result_stage) { assert(!result_stage->parents.empty()); - Block header; - const Stage * parent_stage = result_stage->parents[0]; - const Node * parent_stage_node = result_stage->parents[0]->node; - if (!parent_stage_node->step) - LOG_DEBUG(log, "Step of parent stage's node is empty."); - else if (!parent_stage_node->step->hasOutputStream()) - LOG_DEBUG(log, "Step of parent stage's node doesn't have output stream."); - else if (!parent_stage_node->step->getOutputStream().header) - LOG_DEBUG(log, "Step of parent stage's node has empty header."); - else + /// Clear query plan tree. + root = nullptr; + + for (const auto parent_stage : result_stage->parents) { - header = parent_stage_node->step->getOutputStream().header; - LOG_DEBUG( - log, - "Take the output stream header of {}: {}, header columns: {}.", - parent_stage_node->step->getName(), - parent_stage_node->step->getStepDescription(), - header.columns()); - } + Block header; + const Node * parent_stage_node = parent_stage->node; + if (!parent_stage_node->step->getOutputStream().header) + LOG_ERROR(log, "Step of parent stage's node has empty header."); + else + { + header = parent_stage_node->step->getOutputStream().header; + LOG_DEBUG( + log, + "Take the output stream header of {}: {}, header columns: {}.", + parent_stage_node->step->getName(), + parent_stage_node->step->getStepDescription(), + header.columns()); + } - auto distributed_source_step = std::make_unique( - header, parent_stage->workers, initial_query_id, result_stage->id, parent_stage->id, *result_stage->workers.front(), context); - reset(); - addStep(std::move(distributed_source_step)); + auto distributed_source_step = std::make_unique( + header, + parent_stage->workers, + initial_query_id, + result_stage->id, + parent_stage->id, + *result_stage->workers.front(), + context); + /// TODO: improve to support adding multiple distributed_source_step. + addStep(std::move(distributed_source_step)); + } { /// Only for debug. LOG_DEBUG( log, "Local plan fragment:\n{}", - debugLocalPlanFragment(initial_query_id, result_stage->id, *result_stage->workers.front(), std::vector{root})); + debugLocalPlanFragment( + initial_query_id, result_stage->id, *result_stage->workers.front(), std::vector{root})); } continue; } @@ -422,14 +412,12 @@ void QueryPlan::scheduleStages(ContextPtr context) query_info.set_query_id(context->generateQueryId()); query_info.set_initial_query_id(initial_query_id); query_info.set_stage_id(stage.id); - if (!stage.parents.empty()) - query_info.set_parent_stage_id(stage.parents[0]->id); - else - query_info.set_parent_stage_id(-1); - - for (const auto & source : stage.sources) + for (const auto parent : stage.parents) { - query_info.add_sources(*source); + clickhouse::grpc::MapEntry entry; + for (const auto & source : parent->workers) + entry.add_sources(*source); + (*query_info.mutable_parent_sources())[parent->id] = entry; } for (const auto & sink : stage.sinks) { @@ -454,9 +442,9 @@ void QueryPlan::buildPlanFragment(ContextPtr context) int my_stage_id = query_distributed_plan_info.stage_id; LOG_DEBUG( log, - "Build plan fragment: stage {} parent stage {}.", + "Build plan fragment: stage {}, has {} parent stages.", my_stage_id, - query_distributed_plan_info.parent_stage_id); + query_distributed_plan_info.parent_sources.size()); /// Get my replica grpc address String my_replica = context->getMacros()->getValue("replica") + ":" + toString(context->getServerPort("grpc_port")); @@ -491,12 +479,14 @@ void QueryPlan::buildPlanFragment(ContextPtr context) { ++stage_id; /// Add a DistributedSourceStep between current node and child node. - if (stage_id == query_distributed_plan_info.parent_stage_id) /// TODO: if(query_distributed_plan_info.parent_id_to_sources.contains(stage_id)) + const auto & it = query_distributed_plan_info.parent_sources.find(stage_id); + if (it != query_distributed_plan_info.parent_sources.end()) { assert(child_node == frame.node->children[frame.visited_children]); auto addStep = [this, &stage_id](QueryPlanStepPtr step, const String & description, Node * & node) { + LOG_DEBUG(log, "Add step: {}, parent stage: {}", step->getName(), stage_id); step->setStepDescription(description); if (!node) nodes.emplace_back(Node{.step = std::move(step)}); @@ -506,15 +496,13 @@ void QueryPlan::buildPlanFragment(ContextPtr context) node->parent = &nodes.back(); } node = &nodes.back(); - LOG_DEBUG(log, "Add step: {}, parent stage: {}", step->getName(), stage_id); }; /// Create DistributedSourceStep. const auto & header = child_node->step->getOutputStream().header; - const auto & sources - = query_distributed_plan_info.sources; // TODO: query_distributed_plan_info.parent_id_to_sources[stage_id]; - auto distributed_source_step - = std::make_unique(header, sources, query_distributed_plan_info.initial_query_id, my_stage_id, stage_id, my_replica, context); + const auto & sources = it->second; + auto distributed_source_step = std::make_unique( + header, sources, query_distributed_plan_info.initial_query_id, my_stage_id, stage_id, my_replica, context); Node * new_node = nullptr; addStep(std::move(distributed_source_step), "", new_node); distributed_source_nodes.emplace_back(new_node); /// For debug @@ -645,22 +633,24 @@ String QueryPlan::debugLocalPlanFragment(const String & query_id, int stage_id, return buf.str(); } -String QueryPlan::debugRemotePlanFragment(const String & query, const String & receiver, const String & query_id, const Stage * stage) +String QueryPlan::debugRemotePlanFragment(const String & query, const String & worker, const String & query_id, const Stage * stage) { WriteBufferFromOwnString buf; buf << "------ Remote Plan Fragment ------\n"; buf << "Query: " << query; buf.write('\n'); - buf << "Receiver: " << receiver; + buf << "Worker: " << worker; buf.write('\n'); - buf << "Fragment ID: " << query_id << "/" << stage->id << "/" << receiver; + buf << "Fragment ID: " << query_id << "/" << stage->id << "/" << worker; buf.write('\n'); - buf << "Sources: "; - for (const auto & source : stage->sources) + buf << "Sources:\n"; + for (const auto parent : stage->parents) { - buf << *source << " "; + buf << " parent stage id: " << parent->id << ", sources: "; + for (const auto & source : parent->workers) + buf << *source << " "; + buf.write('\n'); } - buf.write('\n'); buf << "Sinks: "; for (const auto & sink : stage->sinks) { diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 44ead12e2378..0c0254facfa6 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -121,8 +121,7 @@ class QueryPlan std::vector parents = {}; /// Previous stages that current stage directly depends on. Stage * child = nullptr; std::vector> workers; /// Replicas that current stage should be executed on. - std::vector> sources; /// Parents' executors. - std::vector> sinks; /// Child's executors. + std::vector> sinks; /// Child's workers. Node * node; /// Current stage's root node. }; diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 606fe4e9a3ce..3812ce6bc9ca 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1156,11 +1156,15 @@ namespace /// Set query plan fragment info if (call_type == CALL_EXECUTE_PLAN_FRAGMENT) { - std::vector> sources, sinks; - for (const auto & source : query_info.sources()) + std::unordered_map>> parent_sources; + for (const auto & parent : query_info.parent_sources()) { - sources.emplace_back(std::make_shared(source)); + std::vector> sources; + for (const auto & source : parent.second.sources()) + sources.emplace_back(std::make_shared(source)); + parent_sources[parent.first] = std::move(sources); } + std::vector> sinks; for (const auto & sink : query_info.sinks()) { sinks.emplace_back(std::make_shared(sink)); @@ -1168,9 +1172,8 @@ namespace Context::QueryPlanFragmentInfo fragmentInfo{ .initial_query_id = query_info.initial_query_id(), .stage_id = query_info.stage_id(), - .parent_stage_id = query_info.parent_stage_id(), .node_id = query_info.node_id(), - .sources = sources, + .parent_sources = std::move(parent_sources), .sinks = sinks }; query_context->setQueryPlanFragmentInfo(std::move(fragmentInfo)); } diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index 8270f0ab9991..88afc7bf088e 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -68,6 +68,10 @@ message Ticket string node_id = 3; } +message MapEntry { + repeated string sources = 1; +} + // Information about a query which a client sends to a ClickHouse server. // The first QueryInfo can set any of the following fields. Extra QueryInfos only add extra data. // In extra QueryInfos only `input_data`, `external_tables`, `next_query_info` and `cancel` fields can be set. @@ -115,9 +119,8 @@ message QueryInfo { string initial_query_id = 19; int32 stage_id = 20; string node_id = 21; - /// TODO use map to represent the relation between source and parent stage id - int32 parent_stage_id = 22; - repeated string sources = 23; + /// The mapping of parent stage id to sources. + map parent_sources = 22; /// Format is "replica_name:grpc_port" repeated string sinks = 24; } From a3ef01e0ee6c4e54fe93f4f36437a6eea432b72e Mon Sep 17 00:00:00 2001 From: madianjun Date: Thu, 16 Dec 2021 17:19:51 +0800 Subject: [PATCH 10/84] Fix quering system tables on single worker. --- src/Processors/QueryPlan/QueryPlan.cpp | 96 +++++++++++++++++++++----- src/Processors/QueryPlan/QueryPlan.h | 4 +- 2 files changed, 81 insertions(+), 19 deletions(-) diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index ea6058996ff0..1f9b9972c879 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -166,16 +166,26 @@ void QueryPlan::checkShuffle(Node * current_node, Node * child_node, CheckShuffl void QueryPlan::buildStages(ContextPtr) { - LOG_DEBUG(log, "Build stages."); + LOG_DEBUG(log, "===> Build stages."); - auto createStage = [this](int id, Stage * parent_stage, Node * node) { - stages.emplace_back(Stage{.id = id, .node = node}); + auto createStage = [this](int id, Stage * parent_stage, Node * root_node, std::stack & leaf_nodes) { + stages.emplace_back(Stage{.id = id, .root_node = root_node}); Stage * new_stage = &stages.back(); if (parent_stage) { new_stage->parents.push_back(parent_stage); parent_stage->child = new_stage; } + + if (root_node) + { + for (int i = 0; !leaf_nodes.empty() && i < root_node->num_leaf_nodes_in_stage; ++i) + { + new_stage->leaf_nodes.emplace_back(leaf_nodes.top()); + leaf_nodes.pop(); + } + } + LOG_DEBUG(log, "Create stage: id: {}, {} parent stages, {} leaf nodes.", id, new_stage->parents.size(), new_stage->leaf_nodes.size()); return new_stage; }; @@ -193,7 +203,9 @@ void QueryPlan::buildStages(ContextPtr) /// Used for creating stage. int stage_id = -1; Node * child_node = nullptr; + Node * leaf_node = nullptr; Stage * last_stage = nullptr; + std::stack leaf_nodes; while (!stack.empty()) { @@ -204,16 +216,32 @@ void QueryPlan::buildStages(ContextPtr) ++frame.visited_children; one_child_is_visited = false; - /// This is shuffle, create a new stage + /// This is shuffle, create a new stage for child_node. CheckShuffleResult result; checkShuffle(frame.node, child_node, result); if (result.is_shuffle) { ++stage_id; - last_stage = createStage(stage_id, last_stage, child_node); + last_stage = createStage(stage_id, last_stage, child_node, leaf_nodes); + + /// After creating new stage, current node will be in another stage, so save current node as a candidate leaf node. + leaf_node = frame.node; + leaf_nodes.push(leaf_node); + frame.node->num_leaf_nodes_in_stage += 1; + } + else + { + frame.node->num_leaf_nodes_in_stage += child_node->num_leaf_nodes_in_stage; } } + if (frame.node->children.empty()) + { + leaf_node = frame.node; + leaf_nodes.push(leaf_node); + frame.node->num_leaf_nodes_in_stage = 1; + } + size_t next_child = frame.visited_children; if (next_child == frame.node->children.size()) { @@ -226,17 +254,18 @@ void QueryPlan::buildStages(ContextPtr) stack.push(Frame{.node = frame.node->children[next_child]}); } - /// At last, append a shuffle for converging data. + /// Currently, child_node is the root node of query plan, create stage for it. ++stage_id; - last_stage = createStage(stage_id, last_stage, child_node); + last_stage = createStage(stage_id, last_stage, child_node, leaf_nodes); - /// Create result stage. + /// Append result stage for converging data. ++stage_id; - result_stage = createStage(stage_id, last_stage, nullptr); + result_stage = createStage(stage_id, last_stage, nullptr, leaf_nodes); } void QueryPlan::scheduleStages(ContextPtr context) { + LOG_DEBUG(log, "===> Schedule stages."); /// Use initial query id to build the plan fragment id. const String & initial_query_id = context->getClientInfo().current_query_id; @@ -245,14 +274,14 @@ void QueryPlan::scheduleStages(ContextPtr context) /// Retrieve all replicas. std::unordered_map replicas = context->getClustersWatcher().getContainer(); - LOG_DEBUG(log, "Schedule stages for query id {} across {} replicas.", initial_query_id, replicas.size()); + LOG_DEBUG(log, "Schedule stages for query id {} across {} workers.", initial_query_id, replicas.size()); std::vector> store_replicas, compute_replicas; for (const auto & replica : replicas) { const auto & replica_info = replica.second; LOG_DEBUG( log, - "Check replica: {} => ({}/{}/{}, {}).", + "Check worker: {} => ({}/{}/{}, {}).", replica.first, replica_info->type, replica_info->group, @@ -268,17 +297,47 @@ void QueryPlan::scheduleStages(ContextPtr context) compute_replicas.emplace_back(std::make_shared(replica_info->address)); } } - LOG_DEBUG(log, "{} store, {} compute.", store_replicas.size(), compute_replicas.size()); + LOG_DEBUG(log, "{} store workers, {} compute workers.", store_replicas.size(), compute_replicas.size()); + + static std::unordered_set system_tables{"SystemClusters", "SystemDatabases", "SystemTables", "SystemColumns", + "SystemDictionaries", "SystemDataSkippingIndices", + "SystemFunctions", "SystemFormats", "SystemTableEngines", + "SystemUsers", "SystemRoles", "SystemGrants", "SystemRoleGrants", + "SystemCurrentRoles", "SystemEnabledRoles", "SystemRowPolicies", "SystemPrivileges", + "SystemQuotas", "SystemQuotaLimits", + "SystemSettings", "SystemSettingsProfiles", "SystemSettingsProfileElements", + "SystemZooKeeper", + "SystemNumbers", "SystemOne", "SystemZeros", + "SystemContributors", "SystemLicenses"}; auto fillStage = [&store_replicas, &compute_replicas, this, &my_replica](Stage * stage) { /// Leaf stage. if (stage->parents.empty()) { + bool is_multi_points_data_source = false; + for (const auto leaf_node : stage->leaf_nodes) + { + /// It's a data source and not a system table. + if (leaf_node->children.empty() && !system_tables.contains(leaf_node->step->getStepDescription())) + { + LOG_DEBUG(log, "Leaf node {}({}) is multi-points data source.", leaf_node->step->getName(), leaf_node->step->getStepDescription()); + is_multi_points_data_source = true; + break; + } + } /// Fill workers. - stage->workers.reserve(store_replicas.size()); - stage->workers.insert(stage->workers.end(), store_replicas.begin(), store_replicas.end()); - /// Leaf stage's sources should be empty. + if (is_multi_points_data_source) + { + LOG_DEBUG(log, "Schedule to {} workers.", store_replicas.size()); + stage->workers.reserve(store_replicas.size()); + stage->workers.insert(stage->workers.end(), store_replicas.begin(), store_replicas.end()); + } + else + { + LOG_DEBUG(log, "Schedule to 1 worker."); + stage->workers.emplace_back(std::make_shared(my_replica)); + } return; } @@ -345,6 +404,7 @@ void QueryPlan::scheduleStages(ContextPtr context) } /// Send plan fragment params. + LOG_DEBUG(log, "===> Send stages."); for (auto & stage : stages) { /// Don't send result stage. @@ -357,7 +417,7 @@ void QueryPlan::scheduleStages(ContextPtr context) for (const auto parent_stage : result_stage->parents) { Block header; - const Node * parent_stage_node = parent_stage->node; + const Node * parent_stage_node = parent_stage->root_node; if (!parent_stage_node->step->getOutputStream().header) LOG_ERROR(log, "Step of parent stage's node has empty header."); else @@ -386,7 +446,7 @@ void QueryPlan::scheduleStages(ContextPtr context) /// Only for debug. LOG_DEBUG( log, - "Local plan fragment:\n{}", + "Result plan fragment:\n{}", debugLocalPlanFragment( initial_query_id, result_stage->id, *result_stage->workers.front(), std::vector{root})); } @@ -442,7 +502,7 @@ void QueryPlan::buildPlanFragment(ContextPtr context) int my_stage_id = query_distributed_plan_info.stage_id; LOG_DEBUG( log, - "Build plan fragment: stage {}, has {} parent stages.", + "===> Build plan fragment: stage {}, has {} parent stages.", my_stage_id, query_distributed_plan_info.parent_sources.size()); diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 0c0254facfa6..18a25ac70c5b 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -111,6 +111,7 @@ class QueryPlan QueryPlanStepPtr step; std::vector children = {}; Node * parent = nullptr; + int num_leaf_nodes_in_stage = 0; /// Number of leaf nodes(including current node and its descendant nodes) in the same stage. }; using Nodes = std::list; @@ -122,7 +123,8 @@ class QueryPlan Stage * child = nullptr; std::vector> workers; /// Replicas that current stage should be executed on. std::vector> sinks; /// Child's workers. - Node * node; /// Current stage's root node. + Node * root_node; /// Current stage's root node. + std::vector leaf_nodes; }; /// Note: do not use vector, otherwise pointers to elements in it will be invalidated when vector increases. From c34c5cedfc7ced1d6ae8f35a7921451de96770ed Mon Sep 17 00:00:00 2001 From: madianjun Date: Fri, 17 Dec 2021 11:29:28 +0800 Subject: [PATCH 11/84] Fix GRPCClient log and exception message --- src/Client/GRPCClient.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Client/GRPCClient.cpp b/src/Client/GRPCClient.cpp index f7ed9884aae0..ccd69dc0a114 100644 --- a/src/Client/GRPCClient.cpp +++ b/src/Client/GRPCClient.cpp @@ -51,7 +51,7 @@ GRPCResult GRPCClient::executePlanFragment(const GRPCQueryInfo & query_info) "Send query info to {} failed, code: {}, plan fragment id: {}.", addr, status.error_code(), - query_info.initial_query_id() + toString(query_info.stage_id()) + query_info.node_id()); + query_info.initial_query_id() + "/" + toString(query_info.stage_id()) + "/" + query_info.node_id()); throw Exception(status.error_message() + ", " + result.exception().display_text(), ErrorCodes::INVALID_GRPC_QUERY_INFO, true); } } @@ -82,7 +82,7 @@ Block GRPCClient::read() addr, result.exception().code(), result.exception().display_text()); - throw Exception(result.exception().display_text(), ErrorCodes::INVALID_GRPC_QUERY_INFO, true); + throw Exception(result.exception().display_text(), ErrorCodes::GRPC_READ_ERROR, true); } if (result.output().size() == 0) @@ -108,20 +108,20 @@ void GRPCClient::cancel() auto status = inner_context->stub->CancelPlanFragment(&ctx, ticket, &result); - auto plan_fragment_id = ticket.query_id() + toString(ticket.stage_id()) + ticket.node_id(); + auto plan_fragment_id = ticket.initial_query_id() + "/" + toString(ticket.stage_id()) + "/" + ticket.node_id(); if (status.ok()) { if (result.cancelled()) - LOG_INFO(log, "Cancel success from node: {}, plan fragment id: {}", addr, plan_fragment_id); + LOG_INFO(log, "Cancel success from {}, plan fragment id: {}", addr, plan_fragment_id); else { - throw Exception("Cancel failed from node: " + addr + ", plan fragment id: " + plan_fragment_id + ", code: " + toString(result.exception().code()) + ", " + result.exception().display_text(), ErrorCodes::GRPC_CANCEL_ERROR, true); + throw Exception("Cancel failed from " + addr + ", plan fragment id: " + plan_fragment_id + ", code: " + toString(result.exception().code()) + ", " + result.exception().display_text(), ErrorCodes::GRPC_CANCEL_ERROR, true); } } else { LOG_ERROR( - log, "Cancel failed from node {}, code: {}, plan fragment id: {}.", addr, status.error_code(), plan_fragment_id); + log, "Cancel failed from {}, code: {}, plan fragment id: {}.", addr, status.error_code(), plan_fragment_id); throw Exception(status.error_message() + ", " + result.exception().display_text(), ErrorCodes::GRPC_CANCEL_ERROR, true); } } From dfe2a489c294c1ef20b5d73ca194e8c42d6a1bdb Mon Sep 17 00:00:00 2001 From: madianjun Date: Fri, 17 Dec 2021 12:27:15 +0800 Subject: [PATCH 12/84] Fix limit with offset --- src/Processors/QueryPlan/LimitStep.h | 5 +++++ src/Processors/QueryPlan/QueryPlan.cpp | 4 ++++ 2 files changed, 9 insertions(+) diff --git a/src/Processors/QueryPlan/LimitStep.h b/src/Processors/QueryPlan/LimitStep.h index b1e0ec095db6..c939c0d28282 100644 --- a/src/Processors/QueryPlan/LimitStep.h +++ b/src/Processors/QueryPlan/LimitStep.h @@ -47,6 +47,11 @@ class LimitStep : public ITransformingStep bool withTies() const { return with_ties; } + void resetLimitAndOffset() { + limit += offset; + offset = 0; + } + private: size_t limit; size_t offset; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 1f9b9972c879..a4f6d806a287 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -596,6 +596,10 @@ void QueryPlan::buildPlanFragment(ContextPtr context) } else if (stage_id == my_stage_id) { + /// If limit step is pushed down, collect (limit + offset) rows. + if (result.child_limit_step) + result.child_limit_step->resetLimitAndOffset(); + root = child_node; { /// Only for debug. From f2617170f70f6a22c2f34724151e4ef9cdc0e418 Mon Sep 17 00:00:00 2001 From: madianjun Date: Fri, 17 Dec 2021 15:26:07 +0800 Subject: [PATCH 13/84] Refactor code --- src/Interpreters/Context.h | 1 - src/Processors/QueryPlan/QueryPlan.cpp | 56 +++++++++++++------------- 2 files changed, 28 insertions(+), 29 deletions(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 2f748dc45d7a..aafc5b14acdb 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -301,7 +301,6 @@ class Context: public std::enable_shared_from_this String initial_query_id; int stage_id; /// Stage that should be executed on this replica. String node_id; /// This is myself replica name and grpc port. -// int parent_stage_id; std::unordered_map>> parent_sources; /// Mapping of parent id and its workers. std::vector> sinks; /// Point to workers that receiving data. }; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index a4f6d806a287..29578da357f7 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -154,6 +154,10 @@ void QueryPlan::reset() void QueryPlan::checkShuffle(Node * current_node, Node * child_node, CheckShuffleResult & result) { + /// Cases: + /// 1. current node is limit step, child node is sort step: no need shuffle. + /// 2. current node is not limit step, child node is sort step: need shuffle. + /// 3. child node is limit step: need shuffle. result.child_sorting_step = dynamic_cast(child_node->step.get()); if (result.child_sorting_step) result.current_limit_step = dynamic_cast(current_node->step.get()); @@ -202,7 +206,7 @@ void QueryPlan::buildStages(ContextPtr) /// Used for creating stage. int stage_id = -1; - Node * child_node = nullptr; + Node * last_node = nullptr; /// Used for marking the current node's child. Node * leaf_node = nullptr; Stage * last_stage = nullptr; std::stack leaf_nodes; @@ -218,11 +222,11 @@ void QueryPlan::buildStages(ContextPtr) /// This is shuffle, create a new stage for child_node. CheckShuffleResult result; - checkShuffle(frame.node, child_node, result); + checkShuffle(frame.node, last_node, result); if (result.is_shuffle) { ++stage_id; - last_stage = createStage(stage_id, last_stage, child_node, leaf_nodes); + last_stage = createStage(stage_id, last_stage, last_node, leaf_nodes); /// After creating new stage, current node will be in another stage, so save current node as a candidate leaf node. leaf_node = frame.node; @@ -231,7 +235,7 @@ void QueryPlan::buildStages(ContextPtr) } else { - frame.node->num_leaf_nodes_in_stage += child_node->num_leaf_nodes_in_stage; + frame.node->num_leaf_nodes_in_stage += last_node->num_leaf_nodes_in_stage; } } @@ -246,7 +250,7 @@ void QueryPlan::buildStages(ContextPtr) if (next_child == frame.node->children.size()) { LOG_DEBUG(log, "Visit step: {}", frame.node->step->getName()); - child_node = frame.node; + last_node = frame.node; one_child_is_visited = true; stack.pop(); } @@ -256,7 +260,7 @@ void QueryPlan::buildStages(ContextPtr) /// Currently, child_node is the root node of query plan, create stage for it. ++stage_id; - last_stage = createStage(stage_id, last_stage, child_node, leaf_nodes); + last_stage = createStage(stage_id, last_stage, last_node, leaf_nodes); /// Append result stage for converging data. ++stage_id; @@ -416,20 +420,15 @@ void QueryPlan::scheduleStages(ContextPtr context) for (const auto parent_stage : result_stage->parents) { - Block header; const Node * parent_stage_node = parent_stage->root_node; - if (!parent_stage_node->step->getOutputStream().header) - LOG_ERROR(log, "Step of parent stage's node has empty header."); - else - { - header = parent_stage_node->step->getOutputStream().header; - LOG_DEBUG( - log, - "Take the output stream header of {}: {}, header columns: {}.", - parent_stage_node->step->getName(), - parent_stage_node->step->getStepDescription(), - header.columns()); - } + const auto & header = parent_stage_node->step->getOutputStream().header; + assert(header); + LOG_DEBUG( + log, + "Take the output stream header of {}: {}, header columns: {}.", + parent_stage_node->step->getName(), + parent_stage_node->step->getStepDescription(), + header.columns()); auto distributed_source_step = std::make_unique( header, @@ -522,7 +521,7 @@ void QueryPlan::buildPlanFragment(ContextPtr context) /// Used for locating the plan fragment. int stage_id = -1; - Node * child_node = nullptr; + Node * last_node = nullptr; std::vector distributed_source_nodes; /// Only for debug while (!stack.empty()) @@ -532,7 +531,7 @@ void QueryPlan::buildPlanFragment(ContextPtr context) if (one_child_is_visited) { CheckShuffleResult result; - checkShuffle(frame.node, child_node, result); + checkShuffle(frame.node, last_node, result); /// This is a shuffle dependency between current node and the last visited child. if (result.is_shuffle) @@ -542,7 +541,7 @@ void QueryPlan::buildPlanFragment(ContextPtr context) const auto & it = query_distributed_plan_info.parent_sources.find(stage_id); if (it != query_distributed_plan_info.parent_sources.end()) { - assert(child_node == frame.node->children[frame.visited_children]); + assert(last_node == frame.node->children[frame.visited_children]); auto addStep = [this, &stage_id](QueryPlanStepPtr step, const String & description, Node * & node) { @@ -559,7 +558,8 @@ void QueryPlan::buildPlanFragment(ContextPtr context) }; /// Create DistributedSourceStep. - const auto & header = child_node->step->getOutputStream().header; + const auto & header = last_node->step->getOutputStream().header; + assert(header); const auto & sources = it->second; auto distributed_source_step = std::make_unique( header, sources, query_distributed_plan_info.initial_query_id, my_stage_id, stage_id, my_replica, context); @@ -578,8 +578,8 @@ void QueryPlan::buildPlanFragment(ContextPtr context) /// If parent stage has limit, add LimitStep if (result.child_limit_step) { - assert(child_node->children.size() == 1); - const SortingStep * grandchild_sorting_step = dynamic_cast(child_node->children[0]->step.get()); + assert(last_node->children.size() == 1); + const SortingStep * grandchild_sorting_step = dynamic_cast(last_node->children[0]->step.get()); if (grandchild_sorting_step) { auto merging_sorted @@ -600,7 +600,7 @@ void QueryPlan::buildPlanFragment(ContextPtr context) if (result.child_limit_step) result.child_limit_step->resetLimitAndOffset(); - root = child_node; + root = last_node; { /// Only for debug. LOG_DEBUG( @@ -620,7 +620,7 @@ void QueryPlan::buildPlanFragment(ContextPtr context) if (next_child == frame.node->children.size()) { LOG_DEBUG(log, "Visit step: {}", frame.node->step->getName()); - child_node = frame.node; + last_node = frame.node; one_child_is_visited = true; stack.pop(); } @@ -632,7 +632,7 @@ void QueryPlan::buildPlanFragment(ContextPtr context) ++stage_id; if (stage_id == my_stage_id) { - root = child_node; + root = last_node; { /// Only for debug. LOG_DEBUG( From 9b3087f8a263485eb7279f991d708c143543f292 Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Tue, 21 Dec 2021 09:46:57 +0000 Subject: [PATCH 14/84] Rollback distributed DDL --- programs/server/Server.cpp | 2 - src/Databases/DatabaseReplicatedWorker.cpp | 2 +- src/Databases/DatabaseReplicatedWorker.h | 2 +- src/Interpreters/DDLTask.cpp | 6 -- src/Interpreters/DDLTask.h | 3 - src/Interpreters/DDLWorker.cpp | 34 ++------ src/Interpreters/DDLWorker.h | 3 +- src/Interpreters/DatabaseCatalog.cpp | 7 -- src/Interpreters/DatabaseCatalog.h | 1 - src/Interpreters/InterpreterAlterQuery.cpp | 62 +------------- src/Interpreters/InterpreterAlterQuery.h | 1 - src/Interpreters/InterpreterCreateQuery.cpp | 81 +------------------ src/Interpreters/InterpreterCreateQuery.h | 1 - src/Interpreters/InterpreterDropQuery.cpp | 35 -------- .../InterpreterKillQueryQuery.cpp | 8 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 41 +++------- src/Interpreters/executeDDLQueryOnCluster.h | 3 +- src/Interpreters/executeQuery.cpp | 10 +-- src/Interpreters/executeQuery.h | 3 +- src/Interpreters/loadMetadata.cpp | 42 +--------- src/Interpreters/loadMetadata.h | 3 - 21 files changed, 29 insertions(+), 321 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index a7406d4c044e..79ab379b4c53 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1154,8 +1154,6 @@ if (ThreadFuzzer::instance().isEffective()) try { - /// New server download metadata from keeper, and add to server. - downloadMetaData(global_context); auto & database_catalog = DatabaseCatalog::instance(); /// We load temporary database first, because projections need it. database_catalog.initializeAndLoadTemporaryDatabase(); diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index b5e847f7ff75..365a5d028165 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -71,7 +71,7 @@ void DatabaseReplicatedDDLWorker::initializeReplication() last_skipped_entry_name.emplace(DDLTaskBase::getLogEntryName(our_log_ptr)); } -String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry, const String & , const String & ) +String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry) { auto zookeeper = getAndSetZooKeeper(); return enqueueQueryImpl(zookeeper, entry, database); diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index bb83fab77b17..773612e403c0 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -23,7 +23,7 @@ class DatabaseReplicatedDDLWorker : public DDLWorker public: DatabaseReplicatedDDLWorker(DatabaseReplicated * db, ContextPtr context_); - String enqueueQuery(DDLLogEntry & entry, const String & meta_path = "", const String & meta_info = "") override; + String enqueueQuery(DDLLogEntry & entry) override; String tryEnqueueAndExecuteEntry(DDLLogEntry & entry, ContextPtr query_context); diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 84d921c63324..cfce6ac94639 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -197,12 +197,6 @@ void DDLTask::setClusterInfo(ContextPtr context, Poco::Logger * log) throw Exception("Received unknown DDL query", ErrorCodes::UNKNOWN_TYPE_OF_QUERY); cluster_name = query_on_cluster->cluster; - /// If cluster name is all, modify the name according to running mode - if (cluster_name == CLUSTER_TYPE_ALL) - { - cluster_name = context->getRunningMode() == Context::RunningMode::COMPUTE ? CLUSTER_TYPE_COMPUTE : CLUSTER_TYPE_STORE; - } - cluster = context->tryGetCluster(cluster_name); if (!cluster) diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index 8dedb5d97597..ee49274707a8 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -40,9 +40,6 @@ struct HostID HostID() = default; - explicit HostID(const String & host_name_, const UInt16 port_) - : host_name(host_name_), port(port_) {} - explicit HostID(const Cluster::Address & address) : host_name(address.host_name), port(address.port) {} diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 0391c94cba76..d7984af30c72 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -519,7 +519,7 @@ bool DDLWorker::tryExecuteQuery(const String & query, DDLTaskBase & task, const auto query_context = task.makeQueryContext(context, zookeeper); if (!task.is_initial_query) query_scope.emplace(query_context); - executeQuery(istr, ostr, !task.is_initial_query, query_context, {}, std::nullopt, false); + executeQuery(istr, ostr, !task.is_initial_query, query_context, {}); if (auto txn = query_context->getZooKeeperMetadataTransaction()) { @@ -1042,7 +1042,7 @@ void DDLWorker::createStatusDirs(const std::string & node_path, const ZooKeeperP } -String DDLWorker::enqueueQuery(DDLLogEntry & entry, const String & meta_path, const String & meta_info) +String DDLWorker::enqueueQuery(DDLLogEntry & entry) { if (entry.hosts.empty()) throw Exception("Empty host list in a distributed DDL task", ErrorCodes::LOGICAL_ERROR); @@ -1051,34 +1051,8 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry, const String & meta_path, co String query_path_prefix = fs::path(queue_dir) / "query-"; zookeeper->createAncestors(query_path_prefix); - String node_path; - if (meta_path.empty()) - { - node_path = zookeeper->create(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential); - } - else - { - /// Atomic operation to create dll and save meta info - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential)); - if (meta_info.empty()) - { - /// Drop - ops.emplace_back(zkutil::makeRemoveRequest(meta_path, -1)); - } - else - { - if(zookeeper->exists(meta_path)) /// Alter - ops.emplace_back(zkutil::makeSetRequest(meta_path, meta_info, -1)); - else /// Create - ops.emplace_back(zkutil::makeCreateRequest(meta_path, meta_info, zkutil::CreateMode::Persistent)); - } - - /// If this fails, then we'll just retry from the start. - auto responses = zookeeper->multi(ops); - node_path = dynamic_cast(*responses[0]).path_created; - } + String node_path = zookeeper->create(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential); if (max_pushed_entry_metric) { String str_buf = node_path.substr(query_path_prefix.length()); @@ -1100,9 +1074,11 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry, const String & meta_path, co { LOG_INFO(log, "An error occurred while creating auxiliary ZooKeeper directories in {} . They will be created later. Error : {}", node_path, getCurrentExceptionMessage(true)); } + return node_path; } + bool DDLWorker::initializeMainThread() { assert(!initialized); diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index c3fa8652c88e..d2b7c9d169de 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -48,8 +48,7 @@ class DDLWorker virtual ~DDLWorker(); /// Pushes query into DDL queue, returns path to created node - /// Synchronize with the meta service if meta_path not empty - virtual String enqueueQuery(DDLLogEntry & entry, const String & meta_path = "", const String & meta_info = ""); + virtual String enqueueQuery(DDLLogEntry & entry); /// Host ID (name:port) for logging purposes /// Note that in each task hosts are identified individually by name:port from initiator server cluster config diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index c1b15564b225..205d7052d0c7 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include "config_core.h" @@ -452,12 +451,6 @@ bool DatabaseCatalog::isTableExist(const DB::StorageID & table_id, ContextPtr co return db && db->isTableExist(table_id.table_name, context_); } -bool DatabaseCatalog::isTableExist(const String & table_path, ContextPtr context_) const -{ - auto zookeeper = context_->getZooKeeper(); - return zookeeper->exists(table_path); -} - void DatabaseCatalog::assertTableDoesntExist(const StorageID & table_id, ContextPtr context_) const { if (isTableExist(table_id, context_)) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 69454bc6ab71..6079553b0255 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -164,7 +164,6 @@ class DatabaseCatalog : boost::noncopyable, WithMutableContext /// For all of the following methods database_name in table_id must be not empty (even for temporary tables). void assertTableDoesntExist(const StorageID & table_id, ContextPtr context) const; bool isTableExist(const StorageID & table_id, ContextPtr context) const; - bool isTableExist(const String & table_path, ContextPtr context) const; bool isDictionaryExist(const StorageID & table_id) const; StoragePtr getTable(const StorageID & table_id, ContextPtr context) const; diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index bc6f5d8553de..729a495987fe 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -15,11 +15,9 @@ #include #include #include -#include #include #include #include -#include #include @@ -35,12 +33,10 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; extern const int NOT_IMPLEMENTED; extern const int TABLE_IS_READ_ONLY; - extern const int CANNOT_ASSIGN_ALTER; } -InterpreterAlterQuery::InterpreterAlterQuery(const ASTPtr & query_ptr_, ContextPtr context_) - : WithContext(context_), query_ptr(query_ptr_), log(&Poco::Logger::get("InterpreterAlterQuery")) +InterpreterAlterQuery::InterpreterAlterQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) { } @@ -134,11 +130,6 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) if (!mutation_commands.empty()) { - if (alter.is_initial) - { - const_cast(alter).cluster = CLUSTER_TYPE_STORE; - return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess()); - } table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate(); table->mutate(mutation_commands, getContext()); @@ -146,11 +137,6 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) if (!partition_commands.empty()) { - if (alter.is_initial) - { - const_cast(alter).cluster = CLUSTER_TYPE_STORE; - return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess()); - } table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, getContext()->getSettingsRef()); auto partition_commands_pipe = table->alterPartition(metadata_snapshot, partition_commands, getContext()); if (!partition_commands_pipe.empty()) @@ -172,52 +158,6 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) } } - if (alter.is_initial && !alter.table.empty()) - { - const_cast(alter).database = alter.database.empty() ? getContext()->getCurrentDatabase() : alter.database; - auto zookeeper = getContext()->getZooKeeper(); - String path = fs::path(DEFAULT_ZOOKEEPER_METADATA_PATH) / alter.database / alter.table; - if (!DatabaseCatalog::instance().isTableExist(path, getContext())) - { - throw Exception("Table " + alter.database + "." + alter.table + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); - } - if(!alter_commands.empty()) - { - String alter_lock_path = fs::path(path) / "lock"; - Coordination::Error code = zookeeper->tryCreate(alter_lock_path, "", zkutil::CreateMode::Ephemeral); - if (code == Coordination::Error::ZNODEEXISTS) - { - throw Exception("Table " + alter.database + "." + alter.table + " is altered by another one, couldn't acquire lock", ErrorCodes::CANNOT_ASSIGN_ALTER); - } - auto alter_lock = zkutil::EphemeralNodeHolder::existing(alter_lock_path, *zookeeper); - - /// Format alter commands to sql - auto formatAlterCommandsToSql = [&alter, &table] (const String & statement, ContextPtr context, AlterCommands & alter_commands) - { - const_cast(alter).cluster.clear(); - StorageInMemoryMetadata metadata = table->getInMemoryMetadata(); - alter_commands.validate(metadata, context); - alter_commands.prepare(metadata); - table->checkAlterIsPossible(alter_commands, context); - StorageInMemoryMetadata new_metadata = table->getInMemoryMetadata(); - alter_commands.apply(new_metadata, context); - ParserCreateQuery parser; - ASTPtr ast = parseQuery( - parser, - statement.data(), - statement.data() + statement.size(), - "in file ", - 0, - context->getSettingsRef().max_parser_depth); - applyMetadataChangesToCreateQuery(ast, new_metadata); - return getObjectDefinitionFromCreateQuery(ast); - }; - String meta_info = formatAlterCommandsToSql(zookeeper->get(path), getContext(), alter_commands); - const_cast(alter).cluster = CLUSTER_TYPE_ALL; - return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess(), path, meta_info); - } - } - if (!alter_commands.empty()) { StorageInMemoryMetadata metadata = table->getInMemoryMetadata(); diff --git a/src/Interpreters/InterpreterAlterQuery.h b/src/Interpreters/InterpreterAlterQuery.h index da57f12be032..9494a400e7b3 100644 --- a/src/Interpreters/InterpreterAlterQuery.h +++ b/src/Interpreters/InterpreterAlterQuery.h @@ -34,7 +34,6 @@ class InterpreterAlterQuery : public IInterpreter, WithContext BlockIO executeToDatabase(const ASTAlterQuery & alter); ASTPtr query_ptr; - Poco::Logger * log; }; } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a60626aa1f32..6d38c55bd62c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -26,7 +26,6 @@ #include #include #include -#include #include #include @@ -94,7 +93,7 @@ namespace ErrorCodes namespace fs = std::filesystem; InterpreterCreateQuery::InterpreterCreateQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) - : WithMutableContext(context_), query_ptr(query_ptr_), log(&Poco::Logger::get("InterpreterCreateQuery")) + : WithMutableContext(context_), query_ptr(query_ptr_) { } @@ -754,13 +753,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const ErrorCodes::INCORRECT_QUERY); if (as_create.storage) - { create.set(create.storage, as_create.storage->ptr()); - if (create.storage->engine->name == "ReplicatedMergeTree") - { - create.storage->engine->arguments.reset(); - } - } else if (as_create.as_table_function) create.as_table_function = as_create.as_table_function->clone(); else @@ -1298,78 +1291,6 @@ BlockIO InterpreterCreateQuery::execute() { FunctionNameNormalizer().visit(query_ptr.get()); auto & create = query_ptr->as(); - String current_database = getContext()->getCurrentDatabase(); - create.database = create.database.empty() ? current_database : create.database; - if (create.is_initial && !this->internal) - { - auto zookeeper = getContext()->getZooKeeper(); - auto meta_path = DEFAULT_ZOOKEEPER_METADATA_PATH; - zookeeper->createIfNotExists(meta_path, String("")); - String path = fs::path(meta_path) / create.database; - if (!create.table.empty()) - { - path = fs::path(path) / create.table; - } - LOG_DEBUG(log, "Meta path: {}", path); - if (DatabaseCatalog::instance().isTableExist(path, getContext())) - { - if (create.if_not_exists) - return {}; - else - { - if (create.table.empty()) - throw Exception("Database " + create.database + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS); - else - throw Exception("Table " + create.database + "." + create.table + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); - } - - } - create.cluster = getContext()->getRunningMode() == Context::RunningMode::COMPUTE ? CLUSTER_TYPE_COMPUTE : CLUSTER_TYPE_STORE; - prepareOnClusterQuery(create, getContext(), create.cluster); - ASTCreateQuery create_tmp = {create}; - String meta_info; - { - /// Format ast to sql - create.attach = true; - create.if_not_exists = false; - create.as_database.clear(); - create.as_table.clear(); - create.is_populate = false; - create.replace_view = false; - create.replace_table = false; - create.create_or_replace = false; - create.format = nullptr; - create.out_file = nullptr; - create.cluster.clear(); - if (create.table.empty()) - { - /// Database - create.database = TABLE_WITH_UUID_NAME_PLACEHOLDER; - if (!create.storage) - { - bool old_style_database = getContext()->getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary; - auto engine = std::make_shared(); - auto storage = std::make_shared(); - engine->name = old_style_database ? "Ordinary" : "Atomic"; - engine->no_empty_args = true; - storage->set(storage->engine, engine); - create.set(create.storage, storage); - } - } - else - { - /// Table - create.database.clear(); - create.table = TABLE_WITH_UUID_NAME_PLACEHOLDER; - } - meta_info = queryToString(create); - } - create = create_tmp; - /// All cluster to execute - create.cluster = CLUSTER_TYPE_ALL; - LOG_DEBUG(log, "DDL query on cluster: {}, create {}.{}", create.cluster, create.database, create.table); - return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess(), path, meta_info); - } if (!create.cluster.empty()) { prepareOnClusterQuery(create, getContext(), create.cluster); diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 14ee3cad3a22..89d27a30555d 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -93,7 +93,6 @@ class InterpreterCreateQuery : public IInterpreter, WithMutableContext void assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const; ASTPtr query_ptr; - Poco::Logger * log; /// Skip safety threshold when loading tables. bool has_force_restore_data_flag = false; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 16da54472213..4fbad7e5471c 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -11,8 +11,6 @@ #include #include #include -#include -#include #include "config_core.h" @@ -24,7 +22,6 @@ # include #endif -namespace fs = std::filesystem; namespace DB { @@ -33,7 +30,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int SYNTAX_ERROR; extern const int UNKNOWN_TABLE; - extern const int UNKNOWN_DATABASE; extern const int NOT_IMPLEMENTED; extern const int INCORRECT_QUERY; extern const int TABLE_IS_READ_ONLY; @@ -54,37 +50,6 @@ InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, ContextMut BlockIO InterpreterDropQuery::execute() { auto & drop = query_ptr->as(); - if (drop.table.empty() && drop.database.empty()) - throw Exception("Nothing to drop, both names are empty", ErrorCodes::LOGICAL_ERROR); - if (drop.is_initial) - { - auto zookeeper = getContext()->getZooKeeper(); - drop.database = drop.database.empty() ? getContext()->getCurrentDatabase() : drop.database; - String path = fs::path(DEFAULT_ZOOKEEPER_METADATA_PATH) / drop.database; - if (!drop.table.empty()) - path = fs::path(path) / drop.table; - - if (!DatabaseCatalog::instance().isTableExist(path, getContext())) - { - if (!drop.if_exists) - { - if (drop.table.empty()) - throw Exception("Database " + drop.database + " doesn't exist.", ErrorCodes::UNKNOWN_DATABASE); - else - throw Exception("Table " + drop.database + "." + drop.table + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); - } - else - return {}; - } - drop.cluster = CLUSTER_TYPE_ALL; - drop.no_delay = true; - if (drop.kind == ASTDropQuery::Kind::Truncate) - { - drop.cluster = CLUSTER_TYPE_STORE; - path = ""; - } - return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccessForDDLOnCluster(), path, ""); - } if (!drop.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccessForDDLOnCluster()); diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 585e288d66d5..2b949266c170 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -195,13 +195,7 @@ class SyncKillQuerySource : public SourceWithProgress BlockIO InterpreterKillQueryQuery::execute() { - auto & query = query_ptr->as(); - - if (query.is_initial) - { - query.cluster = CLUSTER_TYPE_ALL; - return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccessForDDLOnCluster()); - } + const auto & query = query_ptr->as(); if (!query.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccessForDDLOnCluster()); diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 368222462499..c3b8cc5c6776 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -59,8 +59,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, c return executeDDLQueryOnCluster(query_ptr, context, AccessRightsElements{query_requires_access}); } -BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, AccessRightsElements && query_requires_access, - const String& meta_path, const String& meta_info) +BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, AccessRightsElements && query_requires_access) { /// Remove FORMAT and INTO OUTFILE if exists ASTPtr query_ptr = query_ptr_->clone(); @@ -85,35 +84,17 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, } } - bool is_reserved_cluster = query->cluster == CLUSTER_TYPE_ALL || query->cluster == CLUSTER_TYPE_STORE || query->cluster == CLUSTER_TYPE_COMPUTE; + query->cluster = context->getMacros()->expand(query->cluster); + ClusterPtr cluster = context->getCluster(query->cluster); DDLWorker & ddl_worker = context->getDDLWorker(); + + /// Enumerate hosts which will be used to send query. + Cluster::AddressesWithFailover shards = cluster->getShardsAddresses(); std::vector hosts; - Cluster::AddressesWithFailover shards; - if (!is_reserved_cluster) - { - query->cluster = context->getMacros()->expand(query->cluster); - ClusterPtr cluster = context->getCluster(query->cluster); - /// Enumerate hosts which will be used to send query. - shards = cluster->getShardsAddresses(); - for (const auto & shard : shards) - { - for (const auto & addr : shard) - hosts.emplace_back(addr); - } - } - else + for (const auto & shard : shards) { - /// Get Hosts from meta service - auto getHostsFromMetaService = [&context, &query](std::vector& hosts){ - std::unordered_map replicas = context->getClustersWatcher().getContainer(); - for (const auto & replica : replicas) - { - const auto & replica_info = replica.second; - if (query->cluster == CLUSTER_TYPE_ALL || (query->cluster == replica_info->type)) - hosts.emplace_back(HostID(replica_info->name, context->getTCPPort())); - } - }; - getHostsFromMetaService(hosts); + for (const auto & addr : shard) + hosts.emplace_back(addr); } if (hosts.empty()) @@ -129,7 +110,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, bool use_local_default_database = false; const String & current_database = context->getCurrentDatabase(); - if (!is_reserved_cluster && need_replace_current_database) + if (need_replace_current_database) { Strings shard_default_databases; for (const auto & shard : shards) @@ -182,7 +163,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, entry.query = queryToString(query_ptr); entry.initiator = ddl_worker.getCommonHostID(); entry.setSettingsIfRequired(context); - String node_path = ddl_worker.enqueueQuery(entry, meta_path, meta_info); + String node_path = ddl_worker.enqueueQuery(entry); return getDistributedDDLStatus(node_path, entry, context); } diff --git a/src/Interpreters/executeDDLQueryOnCluster.h b/src/Interpreters/executeDDLQueryOnCluster.h index 822521692b14..e7ec52d03cbf 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.h +++ b/src/Interpreters/executeDDLQueryOnCluster.h @@ -25,8 +25,7 @@ bool isSupportedAlterType(int type); /// Returns DDLQueryStatusSource, which reads results of query execution on each host in the cluster. BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context); BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, const AccessRightsElements & query_requires_access); -BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, AccessRightsElements && query_requires_access, - const String& meta_path = "", const String& meta_info = ""); +BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, AccessRightsElements && query_requires_access); BlockIO getDistributedDDLStatus( const String & node_path, const DDLLogEntry & entry, ContextPtr context, const std::optional & hosts_to_wait = {}); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index d12431afc0a7..ee61ac9198f3 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -376,8 +376,7 @@ static std::tuple executeQueryImpl( ContextMutablePtr context, bool internal, QueryProcessingStage::Enum stage, - ReadBuffer * istr, - bool is_initial = true) + ReadBuffer * istr) { const auto current_time = std::chrono::system_clock::now(); @@ -598,8 +597,6 @@ static std::tuple executeQueryImpl( return std::make_tuple(ast, std::move(io)); } - // is_initial flag - ast->is_initial = is_initial; auto interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); @@ -981,8 +978,7 @@ void executeQuery( bool allow_into_outfile, ContextMutablePtr context, SetResultDetailsFunc set_result_details, - const std::optional & output_format_settings, - bool is_initial) + const std::optional & output_format_settings) { PODArray parse_buf; const char * begin; @@ -1016,7 +1012,7 @@ void executeQuery( ASTPtr ast; BlockIO streams; - std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr, is_initial); + std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr); auto & pipeline = streams.pipeline; std::unique_ptr compressed_buffer; diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index f96cc1fdcf58..9c561d8b88cb 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -20,8 +20,7 @@ void executeQuery( bool allow_into_outfile, /// If true and the query contains INTO OUTFILE section, redirect output to that file. ContextMutablePtr context, /// DB, tables, data types, storage engines, functions, aggregate functions... SetResultDetailsFunc set_result_details, /// If a non-empty callback is passed, it will be called with the query id, the content-type, the format, and the timezone. - const std::optional & output_format_settings = std::nullopt, /// Format settings for output format, will be calculated from the context if not set. - bool is_initial = true /// is initial flag + const std::optional & output_format_settings = std::nullopt /// Format settings for output format, will be calculated from the context if not set. ); diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 06cd6550b8d4..65b2065b2adf 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -9,8 +9,7 @@ #include #include #include -#include -#include + #include #include @@ -167,9 +166,7 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam TablesLoader::Databases loaded_databases; for (const auto & [name, db_path] : databases) { - auto db = DatabaseCatalog::instance().tryGetDatabase(name); - if (!db) - loadDatabase(context, name, db_path, has_force_restore_data_flag); + loadDatabase(context, name, db_path, has_force_restore_data_flag); loaded_databases.insert({name, DatabaseCatalog::instance().getDatabase(name)}); } @@ -234,39 +231,4 @@ void loadMetadataSystem(ContextMutablePtr context) /// Will startup tables in system database after all databases are loaded. } -void downloadMetaData(ContextMutablePtr context) -{ - auto zookeeper = context->getZooKeeper(); - auto meta_server_path = DEFAULT_ZOOKEEPER_METADATA_PATH; - zookeeper->createAncestors(fs::path(meta_server_path) / "default" / ""); - auto meta_local_path = fs::path(context->getPath()) / "metadata"; - auto database_names = zookeeper->getChildren(meta_server_path); - auto write_to_file = [](const String& file, const String& content){ - WriteBufferFromFile out(file, content.size(), O_WRONLY | O_CREAT | O_TRUNC); - writeString(content, out); - out.next(); - out.close(); - }; - for(auto& database : database_names) - { - String local_db_path = fs::path(meta_local_path)/database; - String meta_db_path = fs::path(meta_server_path)/database; - String attach_db = zookeeper->get(meta_db_path); - if (attach_db.empty()) - continue; - write_to_file(local_db_path + ".sql", attach_db); - loadDatabase(context, database, local_db_path, true); - auto table_names = zookeeper->getChildren(meta_db_path); - for (auto& table : table_names) - { - String local_tb_path = fs::path(local_db_path)/table; - local_tb_path += ".sql"; - auto attach_table = zookeeper->get(fs::path(meta_db_path)/table); - if (attach_table.empty()) - continue; - write_to_file(local_tb_path, attach_table); - } - } -} - } diff --git a/src/Interpreters/loadMetadata.h b/src/Interpreters/loadMetadata.h index c13f22ac9ab7..e918b5f530c7 100644 --- a/src/Interpreters/loadMetadata.h +++ b/src/Interpreters/loadMetadata.h @@ -19,7 +19,4 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam /// so we startup system tables after all databases are loaded. void startupSystemTables(); -/// Synchronize databases and tables from meta service -void downloadMetaData(ContextMutablePtr context); - } From 7bdc8e71b42ceea947e05dc763f3b5fa48d24d34 Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Fri, 24 Dec 2021 05:47:50 +0000 Subject: [PATCH 15/84] Support automatical execution on all servers of CREATE/DROP replicated database engine --- src/Core/Settings.h | 2 +- src/Databases/DatabaseReplicated.cpp | 11 ++++++++ src/Databases/DatabaseReplicated.h | 2 ++ src/Interpreters/DDLTask.cpp | 5 ++-- src/Interpreters/InterpreterCreateQuery.cpp | 30 ++++++++++++++++----- src/Interpreters/InterpreterDropQuery.cpp | 25 +++++++++++++++-- src/Interpreters/InterpreterRenameQuery.cpp | 4 +++ src/Interpreters/loadMetadata.cpp | 13 +++++++++ 8 files changed, 80 insertions(+), 12 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ace2ccb9dcd6..8ae6a3691100 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -482,7 +482,7 @@ class IColumn; \ M(Bool, engine_file_empty_if_not_exists, false, "Allows to select data from a file engine table without file", 0) \ M(Bool, engine_file_truncate_on_insert, false, "Enables or disables truncate before insert in file engine tables", 0) \ - M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ + M(Bool, allow_experimental_database_replicated, true, "Allow to create databases with Replicated engine", 0) \ M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \ M(UInt64, max_distributed_depth, 5, "Maximum distributed query depth", 0) \ M(Bool, database_replicated_always_detach_permanently, false, "Execute DETACH TABLE as DETACH TABLE PERMANENTLY if database engine is Replicated", 0) \ diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 3f253ed0924e..319ce269b4f5 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -349,6 +349,8 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_ /// Replicas will set correct name of current database in query context (database name can be different on replicas) if (auto * ddl_query = dynamic_cast(query.get())) { + if (ddl_query->table.empty()) + return; if (ddl_query->database != getDatabaseName()) throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed"); ddl_query->database.clear(); @@ -819,6 +821,15 @@ void DatabaseReplicated::commitCreateTable(const ASTCreateQuery & query, const S DatabaseAtomic::commitCreateTable(query, table, table_metadata_tmp_path, table_metadata_path, query_context); } +void DatabaseReplicated::commitDatabase(ContextPtr query_context) +{ + auto txn = query_context->getZooKeeperMetadataTransaction(); + if (txn && txn->isInitialQuery()) + { + txn->commit(); + } +} + void DatabaseReplicated::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & statement, ContextPtr query_context) diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 34cfb7df1512..6fd0f378c255 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -36,6 +36,8 @@ class DatabaseReplicated : public DatabaseAtomic void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, const String & table_metadata_tmp_path, const String & table_metadata_path, ContextPtr query_context) override; + + void commitDatabase(ContextPtr query_context); void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & statement, ContextPtr query_context) override; diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index cfce6ac94639..3d8297c1c1a2 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -350,9 +350,8 @@ void DatabaseReplicatedTask::parseQueryFromEntry(ContextPtr context) DDLTaskBase::parseQueryFromEntry(context); if (auto * ddl_query = dynamic_cast(query.get())) { - /// Update database name with actual name of local database - assert(ddl_query->database.empty()); - ddl_query->database = database->getDatabaseName(); + if (!ddl_query->table.empty()) + ddl_query->database = database->getDatabaseName(); } } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6d38c55bd62c..caabeca4d6b2 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -132,19 +132,22 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) create.attach_short_syntax = true; create.database = database_name; } - else if (!create.storage) + else if (!create.storage || create.storage->engine->name == "Replicated") { /// For new-style databases engine is explicitly specified in .sql /// When attaching old-style database during server startup, we must always use Ordinary engine - if (create.attach) + if (!create.storage && create.attach) throw Exception("Database engine must be specified for ATTACH DATABASE query", ErrorCodes::UNKNOWN_DATABASE_ENGINE); - bool old_style_database = getContext()->getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary; auto engine = std::make_shared(); auto storage = std::make_shared(); - engine->name = old_style_database ? "Ordinary" : "Atomic"; - engine->no_empty_args = true; + auto args = std::make_shared(); + args->children.emplace_back(std::make_shared(String(fs::path(DEFAULT_ZOOKEEPER_METADATA_PATH) / create.database))); + args->children.emplace_back(std::make_shared(String("{shard}"))); + args->children.emplace_back(std::make_shared(String("{replica}"))); + engine->name = "Replicated"; + engine->arguments = args; storage->set(storage->engine, engine); - create.set(create.storage, storage); + create.setOrReplace(create.storage, storage); } else if ((create.columns_list && ((create.columns_list->indices && !create.columns_list->indices->children.empty()) @@ -234,8 +237,23 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) "Enable allow_experimental_database_materialized_postgresql to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); } + if (!getContext()->getClientInfo().is_replicated_database_internal && !internal) + { + auto * ptr = typeid_cast( + DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); + guard.reset(); + return ptr->tryEnqueueReplicatedDDL(query_ptr, getContext()); + } + DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", getContext()); + if (getContext()->getClientInfo().is_replicated_database_internal && !internal) + { + auto * ptr = typeid_cast( + DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); + ptr->commitDatabase(getContext()); + } + if (create.uuid != UUIDHelpers::Nil) create.database = TABLE_WITH_UUID_NAME_PLACEHOLDER; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 4fbad7e5471c..a1a7bc56af57 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -270,9 +270,15 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name, BlockIO InterpreterDropQuery::executeToDatabase(const ASTDropQuery & query) { + if (query.database == getContext()->getConfigRef().getString("default_database", "default")) + { + throw Exception("The drop/truncate/detach operation is not allowed on default database", ErrorCodes::LOGICAL_ERROR); + } + DatabasePtr database; std::vector tables_to_wait; BlockIO res; + try { res = executeToDatabaseImpl(query, database, tables_to_wait); @@ -312,8 +318,23 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, bool drop = query.kind == ASTDropQuery::Kind::Drop; getContext()->checkAccess(AccessType::DROP_DATABASE, database_name); - if (query.kind == ASTDropQuery::Kind::Detach && query.permanently) - throw Exception("DETACH PERMANENTLY is not implemented for databases", ErrorCodes::NOT_IMPLEMENTED); + if (query.kind == ASTDropQuery::Kind::Detach) + throw Exception("DETACH is not implemented for databases", ErrorCodes::NOT_IMPLEMENTED); + + if (!getContext()->getClientInfo().is_replicated_database_internal) + { + auto * ptr = typeid_cast( + DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); + ddl_guard.reset(); + return ptr->tryEnqueueReplicatedDDL(query_ptr, getContext()); + } + + if (getContext()->getClientInfo().is_replicated_database_internal) + { + auto * ptr = typeid_cast( + DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); + ptr->commitDatabase(getContext()); + } #if USE_MYSQL if (database->getEngineName() == "MaterializedMySQL") diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index e3d52487a52d..5c604034ec01 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -27,6 +27,10 @@ InterpreterRenameQuery::InterpreterRenameQuery(const ASTPtr & query_ptr_, Contex BlockIO InterpreterRenameQuery::execute() { const auto & rename = query_ptr->as(); + if (rename.database) + { + throw Exception("RENAME database is not implemented yet", ErrorCodes::LOGICAL_ERROR); + } if (!rename.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess()); diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 65b2065b2adf..675f40f6c12f 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -1,4 +1,5 @@ #include +#include #include @@ -163,6 +164,18 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam if (create_default_db_if_not_exists && !metadata_dir_for_default_db_already_exists) databases.emplace(default_database_name, std::filesystem::path(path) / escapeForFileName(default_database_name)); + /// Load databases from metaService + { + auto zookeeper = context->getZooKeeper(); + zookeeper->createAncestors(std::filesystem::path(DEFAULT_ZOOKEEPER_METADATA_PATH) / ""); + auto db_names = zookeeper->getChildren(DEFAULT_ZOOKEEPER_METADATA_PATH); + for (auto & db_name : db_names) + { + if (!databases.contains(db_name)) + databases.emplace(db_name, std::filesystem::path(path) / escapeForFileName(db_name)); + } + } + TablesLoader::Databases loaded_databases; for (const auto & [name, db_path] : databases) { From 9d4a8964e73cf9a3851ed8f857cd6f898995b8b0 Mon Sep 17 00:00:00 2001 From: madianjun Date: Fri, 17 Dec 2021 19:06:08 +0800 Subject: [PATCH 16/84] Add distributed aggregate --- src/Processors/QueryPlan/AggregatingStep.h | 6 ++ .../QueryPlan/DistributedSourceStep.cpp | 4 +- .../QueryPlan/DistributedSourceStep.h | 2 + src/Processors/QueryPlan/QueryPlan.cpp | 91 +++++++++++++++++-- src/Processors/QueryPlan/QueryPlan.h | 2 + src/Processors/Sources/DistributedSource.cpp | 23 ++++- src/Processors/Sources/DistributedSource.h | 4 +- src/Server/GRPCServer.cpp | 37 +++++--- 8 files changed, 141 insertions(+), 28 deletions(-) diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 154ff0abc0ad..c16d916f1ed7 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -26,6 +26,12 @@ class AggregatingStep : public ITransformingStep InputOrderInfoPtr group_by_info_, SortDescription group_by_sort_description_); + AggregatingStep(const AggregatingStep & aggregating_step): AggregatingStep(aggregating_step.input_streams.front(), aggregating_step.getParams(), false, + aggregating_step.max_block_size, aggregating_step.aggregation_in_order_max_block_bytes, + aggregating_step.merge_threads, aggregating_step.temporary_data_merge_threads, + aggregating_step.storage_has_evenly_distributed_read, aggregating_step.group_by_info, + aggregating_step.group_by_sort_description) {} + String getName() const override { return "Aggregating"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; diff --git a/src/Processors/QueryPlan/DistributedSourceStep.cpp b/src/Processors/QueryPlan/DistributedSourceStep.cpp index db2c1b2143f2..564cb534019e 100644 --- a/src/Processors/QueryPlan/DistributedSourceStep.cpp +++ b/src/Processors/QueryPlan/DistributedSourceStep.cpp @@ -12,6 +12,7 @@ DistributedSourceStep::DistributedSourceStep( int stage_id_, int parent_stage_id_, const String & node_id_, + bool add_aggregation_info_, ContextPtr context_) : ISourceStep(DataStream{.header = std::move(header_)}) , header(output_stream->header) @@ -20,6 +21,7 @@ DistributedSourceStep::DistributedSourceStep( , stage_id(stage_id_) , parent_stage_id(parent_stage_id_) , node_id(node_id_) + , add_aggregation_info(add_aggregation_info_) , context(std::move(context_)) , log(&Poco::Logger::get("DistributedSourceStep(" + query_id + "/" + toString(stage_id) + "/" + node_id + ")")) { @@ -29,7 +31,7 @@ DistributedSourceStep::DistributedSourceStep( void DistributedSourceStep::addPipe(Pipes & pipes, const std::shared_ptr & source) { auto distributed_source_executor = std::make_shared(header, source, query_id, node_id, parent_stage_id); - pipes.emplace_back(createDistributedSourcePipe(distributed_source_executor, false)); + pipes.emplace_back(createDistributedSourcePipe(distributed_source_executor, add_aggregation_info, false)); pipes.back().addInterpreterContext(context); } diff --git a/src/Processors/QueryPlan/DistributedSourceStep.h b/src/Processors/QueryPlan/DistributedSourceStep.h index f19c4fe6ffc0..c2d1dd001fac 100644 --- a/src/Processors/QueryPlan/DistributedSourceStep.h +++ b/src/Processors/QueryPlan/DistributedSourceStep.h @@ -15,6 +15,7 @@ class DistributedSourceStep final : public ISourceStep int stage_id_, int parent_stage_id_, const String & node_id_, + bool add_aggregation_info_, ContextPtr context_); String getName() const override { return "DistributedSourceStep(" + toString(stage_id) + " <= " + toString(parent_stage_id) + ")"; } @@ -31,6 +32,7 @@ class DistributedSourceStep final : public ISourceStep int stage_id; int parent_stage_id; String node_id; + bool add_aggregation_info; ContextPtr context; Poco::Logger * log; }; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 29578da357f7..56c8bcf519fc 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -11,6 +11,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -155,14 +158,30 @@ void QueryPlan::reset() void QueryPlan::checkShuffle(Node * current_node, Node * child_node, CheckShuffleResult & result) { /// Cases: - /// 1. current node is limit step, child node is sort step: no need shuffle. - /// 2. current node is not limit step, child node is sort step: need shuffle. - /// 3. child node is limit step: need shuffle. + /// 1. child node is aggregate step. + /// 2. current node is limit step, child node is sort step: no need shuffle. + /// 3. current node is not limit step, child node is sort step: need shuffle. + /// 4. child node is limit step: need shuffle. + result.child_aggregating_step = dynamic_cast(child_node->step.get()); + if (result.child_aggregating_step) + { + LOG_DEBUG(log, "Check shuffle: child node is AggregatingStep"); + result.is_shuffle = true; + return; + } + result.child_sorting_step = dynamic_cast(child_node->step.get()); if (result.child_sorting_step) + { + LOG_DEBUG(log, "Check shuffle: child node is SortingStep"); result.current_limit_step = dynamic_cast(current_node->step.get()); + } else + { result.child_limit_step = dynamic_cast(child_node->step.get()); + if (result.child_limit_step) + LOG_DEBUG(log, "Check shuffle: child node is LimitStep"); + } if ((result.child_sorting_step && !result.current_limit_step) || result.child_limit_step) result.is_shuffle = true; @@ -302,6 +321,8 @@ void QueryPlan::scheduleStages(ContextPtr context) } } LOG_DEBUG(log, "{} store workers, {} compute workers.", store_replicas.size(), compute_replicas.size()); + if (store_replicas.empty() || compute_replicas.empty()) + throw Exception("No enough store workers({}) or compute workers({}).", store_replicas.size(), compute_replicas.size()); static std::unordered_set system_tables{"SystemClusters", "SystemDatabases", "SystemTables", "SystemColumns", "SystemDictionaries", "SystemDataSkippingIndices", @@ -437,6 +458,7 @@ void QueryPlan::scheduleStages(ContextPtr context) result_stage->id, parent_stage->id, *result_stage->workers.front(), + false, context); /// TODO: improve to support adding multiple distributed_source_step. addStep(std::move(distributed_source_step)); @@ -537,12 +559,14 @@ void QueryPlan::buildPlanFragment(ContextPtr context) if (result.is_shuffle) { ++stage_id; - /// Add a DistributedSourceStep between current node and child node. + + /// This is one of my parent stages. const auto & it = query_distributed_plan_info.parent_sources.find(stage_id); if (it != query_distributed_plan_info.parent_sources.end()) { assert(last_node == frame.node->children[frame.visited_children]); + /// Add steps between current node and child node. auto addStep = [this, &stage_id](QueryPlanStepPtr step, const String & description, Node * & node) { LOG_DEBUG(log, "Add step: {}, parent stage: {}", step->getName(), stage_id); @@ -557,16 +581,44 @@ void QueryPlan::buildPlanFragment(ContextPtr context) node = &nodes.back(); }; + bool add_agg_info = false; + std::unique_ptr aggregating_step; + if (result.child_aggregating_step) + { + add_agg_info = true; + /// Create AggregatingStep, and it should be non-final. + aggregating_step = std::make_unique(*result.child_aggregating_step); + } + /// The aggregating_step header will include aggregate function. + const auto & header = result.child_aggregating_step == nullptr ? last_node->step->getOutputStream().header + : aggregating_step->getOutputStream().header; + /// Create DistributedSourceStep. - const auto & header = last_node->step->getOutputStream().header; assert(header); const auto & sources = it->second; auto distributed_source_step = std::make_unique( - header, sources, query_distributed_plan_info.initial_query_id, my_stage_id, stage_id, my_replica, context); + header, sources, query_distributed_plan_info.initial_query_id, my_stage_id, stage_id, my_replica, add_agg_info, context); Node * new_node = nullptr; addStep(std::move(distributed_source_step), "", new_node); distributed_source_nodes.emplace_back(new_node); /// For debug + /// If parent stage has aggregate, add MergingAggregatedStep. + if (result.child_aggregating_step) + { + const auto & settings = context->getSettingsRef(); + auto transform_params = std::make_shared(aggregating_step->getParams(), true); + transform_params->params.intermediate_header = new_node->step->getOutputStream().header; + + auto merging_aggregated = std::make_unique( + new_node->step->getOutputStream(), + std::move(transform_params), + settings.distributed_aggregation_memory_efficient, + settings.max_threads, + settings.aggregation_memory_efficient_merge_threads); + + addStep(std::move(merging_aggregated), "Merge aggregated streams for distributed AGGREGATE", new_node); + } + /// If parent stage has order by, add SortingStep if (result.child_sorting_step) { @@ -596,6 +648,20 @@ void QueryPlan::buildPlanFragment(ContextPtr context) } else if (stage_id == my_stage_id) { + auto replaceStep = [this, &stage_id](QueryPlanStepPtr step, Node * & node) + { + LOG_DEBUG(log, "Replace step: {}, stage: {}", step->getName(), stage_id); + node->step = std::move(step); + }; + + if (result.child_aggregating_step) + { + auto aggregating_step = std::make_unique(*result.child_aggregating_step); + { + replaceStep(std::move(aggregating_step), last_node); + } + } + /// If limit step is pushed down, collect (limit + offset) rows. if (result.child_limit_step) result.child_limit_step->resetLimitAndOffset(); @@ -661,6 +727,15 @@ void QueryPlan::buildDistributedPlan(ContextPtr context) LOG_DEBUG(log, "Initial query id: {}, to be built to distributed plan.", context->getInitialQueryId()); + { + /// Print the original query plan. + WriteBufferFromOwnString buf; + buf << "------ Query Plan ------\n"; + QueryPlan::ExplainPlanOptions options{.header = true, .actions = true}; + explainPlan(buf, options); + LOG_DEBUG(log, "After buildQueryPlan:\n{}", buf.str()); + } + checkInitialized(); if (context->isInitialNode()) { @@ -677,7 +752,6 @@ void QueryPlan::buildDistributedPlan(ContextPtr context) String QueryPlan::debugLocalPlanFragment(const String & query_id, int stage_id, const String & node_id, const std::vector distributed_source_nodes) { WriteBufferFromOwnString buf; - ExplainPlanOptions options; buf << "------ Local Plan Fragment ------\n"; buf << "Fragment ID: " << query_id << "/" << stage_id << "/" << node_id; buf.write('\n'); @@ -692,7 +766,8 @@ String QueryPlan::debugLocalPlanFragment(const String & query_id, int stage_id, buf << *source << " "; buf.write('\n'); } - buf << "Plan Fragment:\n"; + buf << "\nPlan Fragment:\n"; + ExplainPlanOptions options{.header = true, .actions = true}; explainPlan(buf, options); return buf.str(); } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 18a25ac70c5b..b33d0eaf5c01 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -27,6 +27,7 @@ using QueryPlanPtr = std::unique_ptr; class Pipe; +class AggregatingStep; class SortingStep; class LimitStep; @@ -144,6 +145,7 @@ class QueryPlan struct CheckShuffleResult { bool is_shuffle = false; + AggregatingStep * child_aggregating_step = nullptr; SortingStep * child_sorting_step = nullptr; LimitStep * current_limit_step = nullptr; LimitStep * child_limit_step = nullptr; diff --git a/src/Processors/Sources/DistributedSource.cpp b/src/Processors/Sources/DistributedSource.cpp index de0f1d37083f..d159f82383e9 100644 --- a/src/Processors/Sources/DistributedSource.cpp +++ b/src/Processors/Sources/DistributedSource.cpp @@ -1,16 +1,23 @@ #include +#include #include +#include #include namespace DB { -DistributedSource::DistributedSource(DistributedSourceExecutorPtr executor, bool async_read_) +DistributedSource::DistributedSource(DistributedSourceExecutorPtr executor, bool add_aggregation_info_, bool async_read_) : SourceWithProgress(executor->getHeader(), false) , query_executor(std::move(executor)) + , add_aggregation_info(add_aggregation_info_) , async_read(async_read_) , log(&Poco::Logger::get("DistributedSource")) { - LOG_DEBUG(log, "DistributedSource header columns: {}.", getPort().getHeader().columns()); + /// Add AggregatedChunkInfo if we expect DataTypeAggregateFunction as a result. + const auto & sample = getPort().getHeader(); + for (auto & type : sample.getDataTypes()) + if (typeid_cast(type.get())) + add_aggregation_info = true; } DistributedSource::~DistributedSource() = default; @@ -62,6 +69,14 @@ std::optional DistributedSource::tryGenerate() UInt64 num_rows = block.rows(); Chunk chunk(block.getColumns(), num_rows); + if (add_aggregation_info) + { + auto info = std::make_shared(); + info->bucket_num = block.info.bucket_num; + info->is_overflows = block.info.is_overflows; + chunk.setChunkInfo(std::move(info)); + } + return std::move(chunk); } @@ -80,9 +95,9 @@ void DistributedSource::onUpdatePorts() } } -Pipe createDistributedSourcePipe(DistributedSourceExecutorPtr query_executor, bool async_read) +Pipe createDistributedSourcePipe(DistributedSourceExecutorPtr query_executor, bool add_aggregation_info, bool async_read) { - Pipe pipe(std::make_shared(query_executor, async_read)); + Pipe pipe(std::make_shared(query_executor, add_aggregation_info, async_read)); return pipe; } diff --git a/src/Processors/Sources/DistributedSource.h b/src/Processors/Sources/DistributedSource.h index 15f123d179cb..dbfbb5e97611 100644 --- a/src/Processors/Sources/DistributedSource.h +++ b/src/Processors/Sources/DistributedSource.h @@ -11,6 +11,7 @@ class DistributedSource : public SourceWithProgress { public: DistributedSource(DistributedSourceExecutorPtr executor, + bool add_aggregation_info_, bool async_read_); ~DistributedSource() override; @@ -26,12 +27,13 @@ class DistributedSource : public SourceWithProgress private: DistributedSourceExecutorPtr query_executor; std::atomic was_query_canceled = false; + bool add_aggregation_info = false; const bool async_read; Poco::Logger * log = nullptr; bool is_async_state = false; }; /// Create pipe with distributed sources. -Pipe createDistributedSourcePipe(DistributedSourceExecutorPtr query_executor, bool async_read); +Pipe createDistributedSourcePipe(DistributedSourceExecutorPtr query_executor, bool add_aggregation_info, bool async_read); } diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 3812ce6bc9ca..46bc5eb2b213 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -904,7 +904,7 @@ namespace void executeQuery(); void storeQueryInfoWrapper(); - void loadQueryInfoWrapper(); + void loadQueryInfoWrapper(bool is_cancel = false); void processInput(); void initializeBlockInputStream(const Block & header); @@ -913,6 +913,7 @@ namespace void generateOutput(); void produceOutput(); void consumeOutput(); + void cancelPlanFragment(); void finishQuery(); void finishQueryInfo(); @@ -1063,8 +1064,9 @@ namespace { setThreadName("GRPCServerCancelPlanFragment"); receiveTicket(); - loadQueryInfoWrapper(); - executeQuery(); + loadQueryInfoWrapper(true); + if (query_info_wrapper) + cancelPlanFragment(); finishQuery(); } else @@ -1118,13 +1120,6 @@ namespace /// and initialize query_context, but don't build pipeline. if (call_type == CALL_FETCH_PLAN_FRAGMENT_RESULT) query_info = *query_info_wrapper->query_info; - else if (call_type == CALL_CANCEL_PLAN_FRAGMENT) - { - /// Find the old executor associated with the ticket, and cancel it - query_info_wrapper->cancel = true; - result.set_cancelled(true); - return; - } /// Retrieve user credentials. std::string user = query_info.user_name(); @@ -1300,13 +1295,20 @@ namespace query_info_wrapper = res.first->second; } - void Call::loadQueryInfoWrapper() + void Call::loadQueryInfoWrapper(bool is_cancel) { query_info_key = ticket.initial_query_id() + "/" + std::to_string(ticket.stage_id()); auto res = query_info_map->get(query_info_key); - if (!res.second) - throw Exception("Query info key " + query_info_key + " not exists", ErrorCodes::LOGICAL_ERROR); - query_info_wrapper = res.first; + if (res.second) + query_info_wrapper = res.first; + else + { + if (is_cancel) /// Plan fragment maybe done. + LOG_INFO(log, "Query info key {} to be cancelled does not exist, so ignore it.", query_info_key); + else + throw Exception("Query info key " + query_info_key + " not exists", ErrorCodes::LOGICAL_ERROR); + } + } void Call::processInput() @@ -1757,6 +1759,13 @@ namespace LOG_DEBUG(log, "{}/{} consumer is {}.", query_info_key, ticket.node_id(), (query_info_wrapper->cancel ? "cancelled" : "done")); } + void Call::cancelPlanFragment() + { + /// Cancel plan fragment, including producer and consumer. + query_info_wrapper->cancel = true; + result.set_cancelled(true); + } + void Call::finishQuery() { finalize = true; From 705c8d54c2b93d080fe344e21aa565d6f73c597e Mon Sep 17 00:00:00 2001 From: madianjun Date: Wed, 22 Dec 2021 17:15:28 +0800 Subject: [PATCH 17/84] Add distributed broadcast join --- src/Core/Settings.h | 1 + .../IInterpreterUnionOrSelectQuery.cpp | 1 + .../QueryPlan/DistributedSourceStep.h | 14 ++ src/Processors/QueryPlan/QueryPlan.cpp | 147 +++++++++++++++--- src/Processors/QueryPlan/QueryPlan.h | 7 +- 5 files changed, 145 insertions(+), 25 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8ae6a3691100..e7309137834c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -263,6 +263,7 @@ class IColumn; M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \ M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \ + M(Bool, enable_distributed_plan, true, "If it is set to true, build distributed plan to execute queries.", 0) \ M(UInt64, query_profiler_real_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(UInt64, query_profiler_cpu_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(Bool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \ diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index 06fe1334094c..be2668c76432 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -19,6 +19,7 @@ QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline() QueryPlan query_plan; buildQueryPlan(query_plan); + context->setSelectQuery(queryToString(this->query_ptr)); query_plan.buildDistributedPlan(context); diff --git a/src/Processors/QueryPlan/DistributedSourceStep.h b/src/Processors/QueryPlan/DistributedSourceStep.h index c2d1dd001fac..0052d34e3fb4 100644 --- a/src/Processors/QueryPlan/DistributedSourceStep.h +++ b/src/Processors/QueryPlan/DistributedSourceStep.h @@ -8,6 +8,20 @@ namespace DB class DistributedSourceStep final : public ISourceStep { public: + /// Used for creating virtual plan node. + DistributedSourceStep(int stage_id_, int parent_stage_id_, ContextPtr context_) + : DistributedSourceStep( + Block{}, + std::vector>{}, + "", + stage_id_, + parent_stage_id_, + "", + false, + context_) + { + } + DistributedSourceStep( Block header_, const std::vector> & sources_, diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 56c8bcf519fc..d5e018001f0c 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -10,7 +10,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -162,6 +164,17 @@ void QueryPlan::checkShuffle(Node * current_node, Node * child_node, CheckShuffl /// 2. current node is limit step, child node is sort step: no need shuffle. /// 3. current node is not limit step, child node is sort step: need shuffle. /// 4. child node is limit step: need shuffle. + result.current_join_step = dynamic_cast(current_node->step.get()); + if (result.current_join_step) + { + LOG_DEBUG(log, "Check shuffle: current node is JoinStep(0x{})", static_cast(result.current_join_step)); + assert(current_node->children.size() == 2); + /// Only broadcast right side. + if (child_node == current_node->children[1]) + result.is_shuffle = true; + return; + } + result.child_aggregating_step = dynamic_cast(child_node->step.get()); if (result.child_aggregating_step) { @@ -187,28 +200,32 @@ void QueryPlan::checkShuffle(Node * current_node, Node * child_node, CheckShuffl result.is_shuffle = true; } -void QueryPlan::buildStages(ContextPtr) +void QueryPlan::buildStages(ContextPtr context) { LOG_DEBUG(log, "===> Build stages."); - auto createStage = [this](int id, Stage * parent_stage, Node * root_node, std::stack & leaf_nodes) { + auto createStage = [this](int id, std::stack & parent_stages, Node * root_node, std::stack & leaf_nodes) { stages.emplace_back(Stage{.id = id, .root_node = root_node}); Stage * new_stage = &stages.back(); - if (parent_stage) - { - new_stage->parents.push_back(parent_stage); - parent_stage->child = new_stage; - } if (root_node) { + for (int i = 0; !parent_stages.empty() && i < root_node->num_parent_stages; ++i) + { + new_stage->parents.emplace_back(parent_stages.top()); + parent_stages.top()->child = new_stage; + parent_stages.pop(); + } for (int i = 0; !leaf_nodes.empty() && i < root_node->num_leaf_nodes_in_stage; ++i) { new_stage->leaf_nodes.emplace_back(leaf_nodes.top()); + /// This leaf node is a data source node reading data from storage. + if (leaf_nodes.top()->children.empty()) + new_stage->is_leaf_stage = true; leaf_nodes.pop(); } } - LOG_DEBUG(log, "Create stage: id: {}, {} parent stages, {} leaf nodes.", id, new_stage->parents.size(), new_stage->leaf_nodes.size()); + LOG_DEBUG(log, "Create stage: id: {}, has {} parent stages and {} leaf nodes.", id, new_stage->parents.size(), new_stage->leaf_nodes.size()); return new_stage; }; @@ -228,6 +245,7 @@ void QueryPlan::buildStages(ContextPtr) Node * last_node = nullptr; /// Used for marking the current node's child. Node * leaf_node = nullptr; Stage * last_stage = nullptr; + std::stack parent_stages; std::stack leaf_nodes; while (!stack.empty()) @@ -236,16 +254,18 @@ void QueryPlan::buildStages(ContextPtr) if (one_child_is_visited) { - ++frame.visited_children; - one_child_is_visited = false; - /// This is shuffle, create a new stage for child_node. CheckShuffleResult result; checkShuffle(frame.node, last_node, result); if (result.is_shuffle) { + ++stage_id; - last_stage = createStage(stage_id, last_stage, last_node, leaf_nodes); + last_stage = createStage(stage_id, parent_stages, last_node, leaf_nodes); + + /// The new stage is parent of current node's stage. + parent_stages.push(last_stage); + frame.node->num_parent_stages += 1; /// After creating new stage, current node will be in another stage, so save current node as a candidate leaf node. leaf_node = frame.node; @@ -254,12 +274,22 @@ void QueryPlan::buildStages(ContextPtr) } else { + frame.node->num_parent_stages += last_node->num_parent_stages; frame.node->num_leaf_nodes_in_stage += last_node->num_leaf_nodes_in_stage; } + + ++frame.visited_children; + one_child_is_visited = false; } if (frame.node->children.empty()) { + if (dynamic_cast(frame.node->step.get())) + throw Exception( + "Not support building distributed plan on Distributed table engine, maybe you want to set " + "enable_distributed_plan=false", + ErrorCodes::LOGICAL_ERROR); + last_stage = nullptr; leaf_node = frame.node; leaf_nodes.push(leaf_node); frame.node->num_leaf_nodes_in_stage = 1; @@ -279,11 +309,75 @@ void QueryPlan::buildStages(ContextPtr) /// Currently, child_node is the root node of query plan, create stage for it. ++stage_id; - last_stage = createStage(stage_id, last_stage, last_node, leaf_nodes); + last_stage = createStage(stage_id, parent_stages, last_node, leaf_nodes); /// Append result stage for converging data. ++stage_id; - result_stage = createStage(stage_id, last_stage, nullptr, leaf_nodes); + /// Create a virtual node, used in iterating stages. + parent_stages.push(last_stage); + auto step = std::make_unique(stage_id, last_stage->id, context); + nodes.emplace_back(Node{.step = std::move(step), .children = {last_node}, .num_parent_stages = 1}); + root = &nodes.back(); + leaf_nodes.push(root); + root->num_leaf_nodes_in_stage = 1; + result_stage = createStage(stage_id, parent_stages, root, leaf_nodes); + + debugStages(); +} + +void QueryPlan::debugStages() +{ + WriteBufferFromOwnString buf; + for (const auto & stage : stages) + { + if (stage.is_leaf_stage) + { + buf << "stage id (leaf) : "; + } + else + { + buf << "stage id (non-leaf) : "; + } + buf << stage.id; + if (stage.child) + { + buf << " => " << stage.child->id; + } + buf.write('\n'); + + buf << "parent stages id : "; + for (const auto parent_stage : stage.parents) + { + buf << parent_stage->id << " "; + } + buf.write('\n'); + + if (stage.root_node) + { + buf << "root node : " << stage.root_node->step->getName(); + buf.write('\n'); + } + + buf << "leaf nodes :\n"; + /// Iterate reversely, because leaf node are stored right to left. + for (auto it = stage.leaf_nodes.rbegin(); it != stage.leaf_nodes.rend(); ++it) + { + buf << " " << (*it)->step->getName(); + if ((*it)->children.empty()) + { + buf << " [S]"; + if (const auto * step = dynamic_cast((*it)->step.get())) + { + const auto & storage_id = step->getStorageID(); + buf << " (" << storage_id.database_name << "." << storage_id.table_name << ")"; + } + } + buf.write('\n'); + } + + buf << "------------------------------\n"; + } + LOG_DEBUG(log, "===> Print Stages:\n{}", buf.str()); } void QueryPlan::scheduleStages(ContextPtr context) @@ -338,7 +432,7 @@ void QueryPlan::scheduleStages(ContextPtr context) auto fillStage = [&store_replicas, &compute_replicas, this, &my_replica](Stage * stage) { /// Leaf stage. - if (stage->parents.empty()) + if (stage->is_leaf_stage) { bool is_multi_points_data_source = false; for (const auto leaf_node : stage->leaf_nodes) @@ -460,7 +554,7 @@ void QueryPlan::scheduleStages(ContextPtr context) *result_stage->workers.front(), false, context); - /// TODO: improve to support adding multiple distributed_source_step. + /// TODO: Improve to support adding multiple distributed_source_step, such as Union operator. addStep(std::move(distributed_source_step)); } { @@ -482,6 +576,7 @@ void QueryPlan::scheduleStages(ContextPtr context) } LOG_DEBUG(log, "Stage {} has {} workers.", stage.id, stage.workers.size()); + assert(!stage.workers.empty()); /// Create query info. GRPCQueryInfo query_info; @@ -622,7 +717,6 @@ void QueryPlan::buildPlanFragment(ContextPtr context) /// If parent stage has order by, add SortingStep if (result.child_sorting_step) { - auto merging_sorted = std::make_unique(new_node->step->getOutputStream(), *result.child_sorting_step); addStep(std::move(merging_sorted), "Merge sorted streams for distributed ORDER BY", new_node); } @@ -712,30 +806,35 @@ void QueryPlan::buildPlanFragment(ContextPtr context) void QueryPlan::buildDistributedPlan(ContextPtr context) { - /// Query directly hit on the store node. + if (!context->getSettingsRef().enable_distributed_plan) + { + LOG_DEBUG(log, "Skip building distributed plan, because enable_distributed_plan=false."); + return; + } + /// Query hits directly on the store worker node. if (context->isInitialNode() && context->getRunningMode() == Context::RunningMode::STORE) { - LOG_DEBUG(log, "Skip building distributed plan."); + LOG_DEBUG(log, "Skip building distributed plan, because initial query hits directly on store worker."); return; } if (context->getInitialQueryId() == "zzzzzzzz-zzzz-zzzz-zzzz-zzzzzzzzzzzz") { - LOG_DEBUG(log, "Special initial query id, skip building distributed plan."); + LOG_DEBUG(log, "Skip building distributed plan, because reserved initial query id is ignored."); return; } - LOG_DEBUG(log, "Initial query id: {}, to be built to distributed plan.", context->getInitialQueryId()); - { /// Print the original query plan. WriteBufferFromOwnString buf; - buf << "------ Query Plan ------\n"; + buf << "------ Original Query Plan ------\n"; QueryPlan::ExplainPlanOptions options{.header = true, .actions = true}; explainPlan(buf, options); - LOG_DEBUG(log, "After buildQueryPlan:\n{}", buf.str()); + LOG_DEBUG(log, "Original query plan:\n{}", buf.str()); } + LOG_DEBUG(log, "Build distributed plan for initial query id: {}.", context->getInitialQueryId()); + checkInitialized(); if (context->isInitialNode()) { diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index b33d0eaf5c01..18c6ad1b2bfe 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -27,6 +27,7 @@ using QueryPlanPtr = std::unique_ptr; class Pipe; +class JoinStep; class AggregatingStep; class SortingStep; class LimitStep; @@ -63,6 +64,7 @@ class QueryPlan void reset(); void buildStages(ContextPtr context); /// Used by initial node. + void debugStages(); void scheduleStages(ContextPtr context); /// Used by initial node. void buildPlanFragment(ContextPtr context); /// Used by non-initial nodes. void buildDistributedPlan(ContextPtr context); @@ -112,6 +114,7 @@ class QueryPlan QueryPlanStepPtr step; std::vector children = {}; Node * parent = nullptr; + int num_parent_stages = 0; /// Number of parent stages whose child is the stage current node belongs to. int num_leaf_nodes_in_stage = 0; /// Number of leaf nodes(including current node and its descendant nodes) in the same stage. }; @@ -125,7 +128,8 @@ class QueryPlan std::vector> workers; /// Replicas that current stage should be executed on. std::vector> sinks; /// Child's workers. Node * root_node; /// Current stage's root node. - std::vector leaf_nodes; + std::vector leaf_nodes; /// Store leaf nodes which are from right side to left side. + bool is_leaf_stage = false; /// Current stage is a leaf stage if it has any leaf node reading data from storage(not from remote). }; /// Note: do not use vector, otherwise pointers to elements in it will be invalidated when vector increases. @@ -145,6 +149,7 @@ class QueryPlan struct CheckShuffleResult { bool is_shuffle = false; + JoinStep * current_join_step = nullptr; AggregatingStep * child_aggregating_step = nullptr; SortingStep * child_sorting_step = nullptr; LimitStep * current_limit_step = nullptr; From 20551acf8bf73429b83b2fd7beef35a43640e317 Mon Sep 17 00:00:00 2001 From: madianjun Date: Thu, 23 Dec 2021 18:38:22 +0800 Subject: [PATCH 18/84] Add distributed union --- src/Processors/QueryPlan/QueryPlan.cpp | 17 ++++++++++++++--- src/Processors/QueryPlan/QueryPlan.h | 2 ++ 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index d5e018001f0c..4a947197b2cb 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -164,6 +165,14 @@ void QueryPlan::checkShuffle(Node * current_node, Node * child_node, CheckShuffl /// 2. current node is limit step, child node is sort step: no need shuffle. /// 3. current node is not limit step, child node is sort step: need shuffle. /// 4. child node is limit step: need shuffle. + result.current_union_step = dynamic_cast(current_node->step.get()); + if (result.current_union_step) + { + LOG_DEBUG(log, "Check shuffle: child node is UnionStep"); + result.is_shuffle = true; + return; + } + result.current_join_step = dynamic_cast(current_node->step.get()); if (result.current_join_step) { @@ -416,7 +425,7 @@ void QueryPlan::scheduleStages(ContextPtr context) } LOG_DEBUG(log, "{} store workers, {} compute workers.", store_replicas.size(), compute_replicas.size()); if (store_replicas.empty() || compute_replicas.empty()) - throw Exception("No enough store workers({}) or compute workers({}).", store_replicas.size(), compute_replicas.size()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No enough store workers({}) or compute workers({}).", store_replicas.size(), compute_replicas.size()); static std::unordered_set system_tables{"SystemClusters", "SystemDatabases", "SystemTables", "SystemColumns", "SystemDictionaries", "SystemDataSkippingIndices", @@ -697,6 +706,8 @@ void QueryPlan::buildPlanFragment(ContextPtr context) addStep(std::move(distributed_source_step), "", new_node); distributed_source_nodes.emplace_back(new_node); /// For debug + /// If current step is JoinStep or UnionStep, only add DistributedSourceStep. + /// If parent stage has aggregate, add MergingAggregatedStep. if (result.child_aggregating_step) { @@ -714,14 +725,14 @@ void QueryPlan::buildPlanFragment(ContextPtr context) addStep(std::move(merging_aggregated), "Merge aggregated streams for distributed AGGREGATE", new_node); } - /// If parent stage has order by, add SortingStep + /// If parent stage has order by, add SortingStep. if (result.child_sorting_step) { auto merging_sorted = std::make_unique(new_node->step->getOutputStream(), *result.child_sorting_step); addStep(std::move(merging_sorted), "Merge sorted streams for distributed ORDER BY", new_node); } - /// If parent stage has limit, add LimitStep + /// If parent stage has limit, add LimitStep. if (result.child_limit_step) { assert(last_node->children.size() == 1); diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 18c6ad1b2bfe..b6df040a8952 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -27,6 +27,7 @@ using QueryPlanPtr = std::unique_ptr; class Pipe; +class UnionStep; class JoinStep; class AggregatingStep; class SortingStep; @@ -149,6 +150,7 @@ class QueryPlan struct CheckShuffleResult { bool is_shuffle = false; + UnionStep * current_union_step = nullptr; JoinStep * current_join_step = nullptr; AggregatingStep * child_aggregating_step = nullptr; SortingStep * child_sorting_step = nullptr; From e35e317731b6308bcd8641777876f37c9fe398ba Mon Sep 17 00:00:00 2001 From: madianjun Date: Fri, 24 Dec 2021 14:27:00 +0800 Subject: [PATCH 19/84] Add exception processing when default database is not Replicated engine --- src/Interpreters/InterpreterCreateQuery.cpp | 2 ++ src/Interpreters/InterpreterDropQuery.cpp | 2 ++ 2 files changed, 4 insertions(+) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index caabeca4d6b2..60445ef327cf 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -242,6 +242,8 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) auto * ptr = typeid_cast( DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); guard.reset(); + if (!ptr) + throw Exception("The default database is not Replicated engine", ErrorCodes::LOGICAL_ERROR); return ptr->tryEnqueueReplicatedDDL(query_ptr, getContext()); } diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index a1a7bc56af57..ade7ae6bae3a 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -326,6 +326,8 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, auto * ptr = typeid_cast( DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); ddl_guard.reset(); + if (!ptr) + throw Exception("The default database is not Replicated engine", ErrorCodes::LOGICAL_ERROR); return ptr->tryEnqueueReplicatedDDL(query_ptr, getContext()); } From decd825aae91c109d852af08c992b0e7b402c2d5 Mon Sep 17 00:00:00 2001 From: madianjun Date: Fri, 24 Dec 2021 15:04:15 +0800 Subject: [PATCH 20/84] Fix bug that doesn't set current database --- src/Processors/QueryPlan/QueryPlan.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 4a947197b2cb..26cf8b5709f7 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -589,6 +589,7 @@ void QueryPlan::scheduleStages(ContextPtr context) /// Create query info. GRPCQueryInfo query_info; + query_info.set_database(context->getCurrentDatabase()); query_info.set_output_format("Native"); if (!context->getSelectQuery().empty()) query_info.set_query(context->getSelectQuery()); /// For "insert into ... select" From c2e77fc5cfe77412a21503849f4e0e51c41384c5 Mon Sep 17 00:00:00 2001 From: Chao Ma Date: Fri, 24 Dec 2021 15:25:27 +0800 Subject: [PATCH 21/84] Add distributed insert for table engine mergetree --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 3 ++- src/Storages/StorageMergeTree.cpp | 9 ++++++++- src/Storages/StorageMergeTree.h | 5 ++++- 3 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 2b7098b8a3e3..163090ea0df7 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -822,7 +822,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) date_column_name, merging_params, std::move(storage_settings), - args.has_force_restore_data_flag); + args.has_force_restore_data_flag, + embedded_distributed); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7322d57fed2f..2ea20f63d31c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -67,7 +67,8 @@ StorageMergeTree::StorageMergeTree( const String & date_column_name, const MergingParams & merging_params_, std::unique_ptr storage_settings_, - bool has_force_restore_data_flag) + bool has_force_restore_data_flag, + StoragePtr embedded_distributed_) : MergeTreeData( table_id_, relative_data_path_, @@ -83,6 +84,7 @@ StorageMergeTree::StorageMergeTree( , merger_mutator(*this, getContext()->getSettingsRef().background_merges_mutations_concurrency_ratio * getContext()->getSettingsRef().background_pool_size) + , embedded_distributed(embedded_distributed_) { loadDataParts(has_force_restore_data_flag); @@ -230,6 +232,11 @@ std::optional StorageMergeTree::totalBytes(const Settings &) const SinkToStoragePtr StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { + if (getContext()->getRunningMode() == Context::RunningMode::COMPUTE) + { + return embedded_distributed->write(nullptr, metadata_snapshot, local_context); + } + const auto & settings = local_context->getSettingsRef(); return std::make_shared( *this, metadata_snapshot, settings.max_partitions_per_insert_block, local_context); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 8ed4b707b347..87f022a051b0 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -137,6 +137,8 @@ class StorageMergeTree final : public shared_ptr_helper, publi std::atomic shutdown_called {false}; + StoragePtr embedded_distributed; + private: void loadMutations(); @@ -236,7 +238,8 @@ class StorageMergeTree final : public shared_ptr_helper, publi const String & date_column_name, const MergingParams & merging_params_, std::unique_ptr settings_, - bool has_force_restore_data_flag); + bool has_force_restore_data_flag, + StoragePtr embedded_distributed_); MutationCommands getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const override; }; From e2d10bb8e134d5df36544f03725c69edc8e1262c Mon Sep 17 00:00:00 2001 From: madianjun Date: Mon, 27 Dec 2021 20:24:05 +0800 Subject: [PATCH 22/84] Add distributed materialized view --- src/Interpreters/Context.cpp | 19 +++++++ src/Interpreters/Context.h | 2 + .../IInterpreterUnionOrSelectQuery.cpp | 6 +-- src/Interpreters/InterpreterSelectQuery.cpp | 6 +-- .../InterpreterSelectWithUnionQuery.cpp | 6 +-- src/Processors/QueryPlan/QueryPlan.cpp | 49 ++++++++++++++----- src/Processors/QueryPlan/QueryPlan.h | 5 +- .../Transforms/buildPushingToViewsChain.cpp | 7 ++- src/Server/GRPCServer.cpp | 13 ++++- src/Server/grpc_protos/clickhouse_grpc.proto | 1 + src/Storages/Distributed/DistributedSink.cpp | 2 +- src/Storages/Kafka/StorageKafka.cpp | 8 +++ 12 files changed, 96 insertions(+), 28 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 83860244a343..aed64709308c 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -149,6 +149,7 @@ struct ContextSharedPart mutable std::mutex external_dictionaries_mutex; mutable std::mutex external_user_defined_executable_functions_mutex; mutable std::mutex external_models_mutex; + mutable std::mutex view_sources_mutex; /// Separate mutex for storage policies. During server startup we may /// initialize some important storages (system logs with MergeTree engine) /// under context lock. @@ -188,6 +189,7 @@ struct ContextSharedPart mutable std::optional external_dictionaries_loader; mutable std::optional external_user_defined_executable_functions_loader; mutable std::optional external_models_loader; + std::unordered_map view_sources; ExternalLoaderXMLConfigRepository * external_models_config_repository = nullptr; scope_guard models_repository_guard; @@ -1057,6 +1059,23 @@ StoragePtr Context::getViewSource() const return view_source; } +void Context::addPlanFragmentViewSource(const String & plan_fragment_id, const StoragePtr & storage) +{ + std::lock_guard lock(shared->view_sources_mutex); + shared->view_sources[plan_fragment_id] = storage; +} + +StoragePtr Context::getPlanFragmentViewSource(const String & plan_fragment_id) const +{ + std::lock_guard lock(shared->view_sources_mutex); + auto it = shared->view_sources.find(plan_fragment_id); + if (it == shared->view_sources.end()) + throw Exception(ErrorCodes::BAD_GET, "There is no view source for {}", plan_fragment_id); + StoragePtr ret_view_source = it->second; + shared->view_sources.erase(it); + return ret_view_source; +} + Settings Context::getSettings() const { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index aafc5b14acdb..07aed4e2432a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -525,6 +525,8 @@ class Context: public std::enable_shared_from_this void addViewSource(const StoragePtr & storage); StoragePtr getViewSource() const; + void addPlanFragmentViewSource(const String & plan_fragment_id, const StoragePtr & storage); + StoragePtr getPlanFragmentViewSource(const String & plan_fragment_id) const; String getCurrentDatabase() const; String getCurrentQueryId() const { return client_info.current_query_id; } diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index be2668c76432..24f1db869de1 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -21,11 +21,11 @@ QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline() buildQueryPlan(query_plan); context->setSelectQuery(queryToString(this->query_ptr)); - query_plan.buildDistributedPlan(context); + bool is_built = query_plan.buildDistributedPlan(context); - QueryPlanOptimizationSettings do_not_optimize_plan{.optimize_plan = false}; + QueryPlanOptimizationSettings optimization_settings{.optimize_plan = !is_built}; return std::move(*query_plan.buildQueryPipeline( - do_not_optimize_plan, BuildQueryPipelineSettings::fromContext(context))); + optimization_settings, BuildQueryPipelineSettings::fromContext(context))); } } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 22f61a71b4c4..004c719c486c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -594,11 +594,11 @@ BlockIO InterpreterSelectQuery::execute() buildQueryPlan(query_plan); - query_plan.buildDistributedPlan(context); + bool is_built = query_plan.buildDistributedPlan(context); - QueryPlanOptimizationSettings do_not_optimize_plan{.optimize_plan = false}; + QueryPlanOptimizationSettings optimization_settings{.optimize_plan = !is_built}; res.pipeline = QueryPipelineBuilder::getPipeline(std::move(*query_plan.buildQueryPipeline( - do_not_optimize_plan, BuildQueryPipelineSettings::fromContext(context)))); + optimization_settings, BuildQueryPipelineSettings::fromContext(context)))); return res; } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 88eb38a402d0..72f70b2a9c42 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -328,11 +328,11 @@ BlockIO InterpreterSelectWithUnionQuery::execute() QueryPlan query_plan; buildQueryPlan(query_plan); - query_plan.buildDistributedPlan(context); + bool is_built = query_plan.buildDistributedPlan(context); - QueryPlanOptimizationSettings do_not_optimize_plan{.optimize_plan = false}; + QueryPlanOptimizationSettings optimization_settings{.optimize_plan = !is_built}; auto pipeline_builder = query_plan.buildQueryPipeline( - do_not_optimize_plan, + optimization_settings, BuildQueryPipelineSettings::fromContext(context)); pipeline_builder->addInterpreterContext(context); diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 26cf8b5709f7..d46b576bc0fb 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -389,7 +389,7 @@ void QueryPlan::debugStages() LOG_DEBUG(log, "===> Print Stages:\n{}", buf.str()); } -void QueryPlan::scheduleStages(ContextPtr context) +void QueryPlan::scheduleStages(ContextMutablePtr context) { LOG_DEBUG(log, "===> Schedule stages."); /// Use initial query id to build the plan fragment id. @@ -446,12 +446,26 @@ void QueryPlan::scheduleStages(ContextPtr context) bool is_multi_points_data_source = false; for (const auto leaf_node : stage->leaf_nodes) { - /// It's a data source and not a system table. - if (leaf_node->children.empty() && !system_tables.contains(leaf_node->step->getStepDescription())) + /// It's a data source. + if (leaf_node->children.empty()) { - LOG_DEBUG(log, "Leaf node {}({}) is multi-points data source.", leaf_node->step->getName(), leaf_node->step->getStepDescription()); - is_multi_points_data_source = true; - break; + if (system_tables.contains(leaf_node->step->getStepDescription())) /// It's system table. + { + } + else if (leaf_node->step->getStepDescription() == "Values") /// It's StorageValues. + { + stage->has_view_source = true; + } + else + { + LOG_DEBUG( + log, + "Leaf node {}({}) is multi-points data source.", + leaf_node->step->getName(), + leaf_node->step->getStepDescription()); + is_multi_points_data_source = true; + break; + } } } /// Fill workers. @@ -598,6 +612,8 @@ void QueryPlan::scheduleStages(ContextPtr context) query_info.set_query_id(context->generateQueryId()); query_info.set_initial_query_id(initial_query_id); query_info.set_stage_id(stage.id); + query_info.set_has_view_source(stage.has_view_source); + for (const auto parent : stage.parents) { clickhouse::grpc::MapEntry entry; @@ -615,6 +631,14 @@ void QueryPlan::scheduleStages(ContextPtr context) query_info.set_node_id(*worker); LOG_DEBUG(log, "Remote plan fragment:\n{}", debugRemotePlanFragment(query_info.query(), *worker, initial_query_id, &stage)); + if (stage.has_view_source) + { + const String & plan_fragment_id + = query_info.initial_query_id() + "/" + toString(query_info.stage_id()) + "/" + query_info.node_id(); + context->addPlanFragmentViewSource(plan_fragment_id, context->getViewSource()); + LOG_DEBUG(log, "Add plan fragment view source."); + } + GRPCClient cli(*worker); auto result = cli.executePlanFragment(query_info); LOG_DEBUG(log, "GRPCClient got result, exception code: {}, exception text: {}.", result.exception().code(), result.exception().display_text()); @@ -816,24 +840,24 @@ void QueryPlan::buildPlanFragment(ContextPtr context) } } -void QueryPlan::buildDistributedPlan(ContextPtr context) +bool QueryPlan::buildDistributedPlan(ContextMutablePtr context) { if (!context->getSettingsRef().enable_distributed_plan) { LOG_DEBUG(log, "Skip building distributed plan, because enable_distributed_plan=false."); - return; + return false; } /// Query hits directly on the store worker node. if (context->isInitialNode() && context->getRunningMode() == Context::RunningMode::STORE) { LOG_DEBUG(log, "Skip building distributed plan, because initial query hits directly on store worker."); - return; + return false; } if (context->getInitialQueryId() == "zzzzzzzz-zzzz-zzzz-zzzz-zzzzzzzzzzzz") { LOG_DEBUG(log, "Skip building distributed plan, because reserved initial query id is ignored."); - return; + return false; } { @@ -845,12 +869,10 @@ void QueryPlan::buildDistributedPlan(ContextPtr context) LOG_DEBUG(log, "Original query plan:\n{}", buf.str()); } - LOG_DEBUG(log, "Build distributed plan for initial query id: {}.", context->getInitialQueryId()); - checkInitialized(); + optimize(QueryPlanOptimizationSettings::fromContext(context)); if (context->isInitialNode()) { - optimize(QueryPlanOptimizationSettings::fromContext(context)); buildStages(context); scheduleStages(context); } @@ -858,6 +880,7 @@ void QueryPlan::buildDistributedPlan(ContextPtr context) { buildPlanFragment(context); } + return true; } String QueryPlan::debugLocalPlanFragment(const String & query_id, int stage_id, const String & node_id, const std::vector distributed_source_nodes) diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index b6df040a8952..57be7849e373 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -66,9 +66,9 @@ class QueryPlan void buildStages(ContextPtr context); /// Used by initial node. void debugStages(); - void scheduleStages(ContextPtr context); /// Used by initial node. + void scheduleStages(ContextMutablePtr context); /// Used by initial node. void buildPlanFragment(ContextPtr context); /// Used by non-initial nodes. - void buildDistributedPlan(ContextPtr context); + bool buildDistributedPlan(ContextMutablePtr context); QueryPipelineBuilderPtr buildQueryPipeline( const QueryPlanOptimizationSettings & optimization_settings, @@ -131,6 +131,7 @@ class QueryPlan Node * root_node; /// Current stage's root node. std::vector leaf_nodes; /// Store leaf nodes which are from right side to left side. bool is_leaf_stage = false; /// Current stage is a leaf stage if it has any leaf node reading data from storage(not from remote). + bool has_view_source = false; /// Current stage reads data to trigger materialized view. }; /// Note: do not use vector, otherwise pointers to elements in it will be invalidated when vector increases. diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index e5abc3bd4164..da30c55fa757 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -163,7 +163,9 @@ Chain buildPushingToViewsChain( disable_deduplication_for_children = !no_destination && storage->supportsDeduplication(); auto table_id = storage->getStorageID(); - Dependencies dependencies = DatabaseCatalog::instance().getDependencies(table_id); + Dependencies dependencies; + if (context->getRunningMode() == Context::RunningMode::COMPUTE) + dependencies = DatabaseCatalog::instance().getDependencies(table_id); /// We need special context for materialized views insertions ContextMutablePtr select_context; @@ -375,6 +377,9 @@ static void process(Block & block, ViewRuntimeData & view, const ViewsData & vie block, views_data.source_storage->getVirtuals())); + /// Set current query id and initial query id. + local_context->getClientInfo().setInitialQuery(); + local_context->setCurrentQueryId(""); /// We need keep InterpreterSelectQuery, until the processing will be finished, since: /// /// - We copy Context inside InterpreterSelectQuery to support diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 46bc5eb2b213..bfb4b6a9a03c 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1164,13 +1164,22 @@ namespace { sinks.emplace_back(std::make_shared(sink)); } - Context::QueryPlanFragmentInfo fragmentInfo{ + Context::QueryPlanFragmentInfo fragment_info{ .initial_query_id = query_info.initial_query_id(), .stage_id = query_info.stage_id(), .node_id = query_info.node_id(), .parent_sources = std::move(parent_sources), .sinks = sinks }; - query_context->setQueryPlanFragmentInfo(std::move(fragmentInfo)); + query_context->setQueryPlanFragmentInfo(std::move(fragment_info)); + + if (query_info.has_view_source()) + { + const String & plan_fragment_id + = fragment_info.initial_query_id + "/" + toString(fragment_info.stage_id) + "/" + fragment_info.node_id; + LOG_DEBUG(log, "Get plan fragment view source for {}", plan_fragment_id); + const auto & view_source = query_context->getPlanFragmentViewSource(plan_fragment_id); + query_context->addViewSource(view_source); + } } /// Prepare settings. diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index 88afc7bf088e..87c5084ff51f 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -123,6 +123,7 @@ message QueryInfo { map parent_sources = 22; /// Format is "replica_name:grpc_port" repeated string sinks = 24; + bool has_view_source = 25; } enum LogsLevel { diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 39857a307865..cc689d9fb27a 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -127,7 +127,7 @@ DistributedSink::DistributedSink( , insert_timeout(insert_timeout_) , main_table(main_table_) , columns_to_send(columns_to_send_.begin(), columns_to_send_.end()) - , log(&Poco::Logger::get("DistributedBlockOutputStream")) + , log(&Poco::Logger::get("DistributedSink(" + main_table.getFullNameNotQuoted() + ")")) { const auto & settings = context->getSettingsRef(); if (settings.max_distributed_depth && context->getClientInfo().distributed_depth > settings.max_distributed_depth) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 25384db043fe..1e9ce72e3af1 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -197,6 +197,10 @@ StorageKafka::StorageKafka( storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); auto task_count = thread_per_consumer ? num_consumers : 1; + /// Consuming is disabled on store workers. + if (getContext()->getRunningMode() == Context::RunningMode::STORE) + task_count = 0; + for (size_t i = 0; i < task_count; ++i) { auto task = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this, i]{ threadFunc(i); }); @@ -299,6 +303,10 @@ SinkToStoragePtr StorageKafka::write(const ASTPtr &, const StorageMetadataPtr & void StorageKafka::startup() { + /// Consuming is disabled on store workers. + if (getContext()->getRunningMode() == Context::RunningMode::STORE) + return; + for (size_t i = 0; i < num_consumers; ++i) { try From 05c69f246cc784431af21da3925c7f28c7a17e16 Mon Sep 17 00:00:00 2001 From: madianjun Date: Thu, 30 Dec 2021 18:30:31 +0800 Subject: [PATCH 23/84] Read data of external tables on initial node --- src/Processors/QueryPlan/QueryPlan.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index d46b576bc0fb..03b7f6a77fcf 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -438,6 +438,8 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) "SystemNumbers", "SystemOne", "SystemZeros", "SystemContributors", "SystemLicenses"}; + static std::unordered_set special_storages{"HDFS", "S3", "MySQL", "Input"}; + auto fillStage = [&store_replicas, &compute_replicas, this, &my_replica](Stage * stage) { /// Leaf stage. @@ -449,10 +451,13 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) /// It's a data source. if (leaf_node->children.empty()) { - if (system_tables.contains(leaf_node->step->getStepDescription())) /// It's system table. + /// It's system table or special storage. + if (system_tables.contains(leaf_node->step->getStepDescription()) || + special_storages.contains(leaf_node->step->getStepDescription())) { } - else if (leaf_node->step->getStepDescription() == "Values") /// It's StorageValues. + /// It's StorageValues. + else if (leaf_node->step->getStepDescription() == "Values") { stage->has_view_source = true; } From 989d1e9212d92ef90b80b48077fe68310ec19434 Mon Sep 17 00:00:00 2001 From: Chao Ma Date: Wed, 29 Dec 2021 16:24:36 +0800 Subject: [PATCH 24/84] Skip distributed data which insert into system database --- src/Core/Settings.h | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e7309137834c..548af37e9f8a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -247,7 +247,7 @@ class IColumn; /** Settings for testing connection collector */ \ M(Milliseconds, sleep_in_receive_cancel_ms, 0, "Time to sleep in receiving cancel in TCPHandler", 0) \ \ - M(Bool, insert_allow_materialized_columns, false, "If setting is enabled, Allow materialized columns in INSERT.", 0) \ + M(Bool, insert_allow_materialized_columns, true, "If setting is enabled, Allow materialized columns in INSERT.", 0) \ M(Seconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \ M(Seconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \ M(Seconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout", 0) \ diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2ea20f63d31c..0d1c0684624d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -232,7 +232,7 @@ std::optional StorageMergeTree::totalBytes(const Settings &) const SinkToStoragePtr StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { - if (getContext()->getRunningMode() == Context::RunningMode::COMPUTE) + if (getContext()->getRunningMode() == Context::RunningMode::COMPUTE && getStorageID().getDatabaseName() != DatabaseCatalog::SYSTEM_DATABASE) { return embedded_distributed->write(nullptr, metadata_snapshot, local_context); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 24911bb0269a..f5e385773931 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4317,7 +4317,7 @@ void StorageReplicatedMergeTree::assertNotReadonly() const SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { - if (getContext()->getRunningMode() == Context::RunningMode::COMPUTE) + if (getContext()->getRunningMode() == Context::RunningMode::COMPUTE && getStorageID().getDatabaseName() != DatabaseCatalog::SYSTEM_DATABASE) { return embedded_distributed->write(nullptr, metadata_snapshot, local_context); } From de9962b2d94e99ead8ddc291b7c08fe49d9b14c5 Mon Sep 17 00:00:00 2001 From: madianjun Date: Tue, 4 Jan 2022 16:50:29 +0800 Subject: [PATCH 25/84] Fix to initialize ClustersWatcher before all servers start --- programs/server/Server.cpp | 6 +++--- src/Interpreters/Cluster.cpp | 4 ++-- src/Interpreters/Cluster.h | 3 +-- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 79ab379b4c53..f3c4952c232a 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1562,13 +1562,13 @@ if (ThreadFuzzer::instance().isEffective()) &CurrentMetrics::MaxDDLEntryID, &CurrentMetrics::MaxPushedDDLEntryID)); } + /// Register with clickhouse-keeper and watch on clusters change. + global_context->setClustersWatcher(std::make_unique(DEFAULT_ZOOKEEPER_CLUSTERS_PATH, global_context)); + for (auto & server : *servers) server.start(); LOG_INFO(log, "Ready for connections."); - /// Register with clickhouse-keeper and watch on clusters change - global_context->setClustersWatcher(std::make_unique(DEFAULT_ZOOKEEPER_CLUSTERS_PATH, global_context, "ClustersWatcher")); - SCOPE_EXIT_SAFE({ LOG_DEBUG(log, "Received termination signal."); LOG_DEBUG(log, "Waiting for current connections to close."); diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index dff4a76bb174..1be0db35dbe2 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -723,8 +723,8 @@ bool Cluster::maybeCrossReplication() const return false; } -ClustersWatcher::ClustersWatcher(const std::string & clusters_path_, ContextPtr context_, const String & logger_name_) - : context(Context::createCopy(context_)), log(&Poco::Logger::get(logger_name_)) +ClustersWatcher::ClustersWatcher(const std::string & clusters_path_, ContextPtr context_) + : context(Context::createCopy(context_)), log(&Poco::Logger::get("ClustersWatcher")) { clusters_path = clusters_path_; if (clusters_path.back() == '/') diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index d6167242884b..f5fcf5cf3660 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -308,8 +308,7 @@ class Clusters class ClustersWatcher { public: - ClustersWatcher(const std::string & clusters_path_, ContextPtr context_, - const String & logger_name_ = "ClustersWatcher"); + ClustersWatcher(const std::string & clusters_path_, ContextPtr context_); void startup(); void shutdown() { task->deactivate(); } From 73290cb47b834cb971f93d92db62859ef454a86d Mon Sep 17 00:00:00 2001 From: madianjun Date: Fri, 31 Dec 2021 11:40:12 +0800 Subject: [PATCH 26/84] Support input function in distributed mode --- src/Interpreters/Context.cpp | 31 ++++++----- src/Interpreters/Context.h | 8 ++- src/Processors/QueryPlan/QueryPlan.cpp | 17 ++++-- src/Processors/QueryPlan/QueryPlan.h | 1 + src/Server/GRPCServer.cpp | 58 ++++++++++++-------- src/Server/grpc_protos/clickhouse_grpc.proto | 1 + src/Storages/StorageInput.cpp | 10 +++- 7 files changed, 79 insertions(+), 47 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index aed64709308c..f3d3f088757d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -149,7 +149,7 @@ struct ContextSharedPart mutable std::mutex external_dictionaries_mutex; mutable std::mutex external_user_defined_executable_functions_mutex; mutable std::mutex external_models_mutex; - mutable std::mutex view_sources_mutex; + mutable std::mutex initial_query_contexts_mutex; /// Separate mutex for storage policies. During server startup we may /// initialize some important storages (system logs with MergeTree engine) /// under context lock. @@ -189,7 +189,7 @@ struct ContextSharedPart mutable std::optional external_dictionaries_loader; mutable std::optional external_user_defined_executable_functions_loader; mutable std::optional external_models_loader; - std::unordered_map view_sources; + std::unordered_map initial_query_contexts; ExternalLoaderXMLConfigRepository * external_models_config_repository = nullptr; scope_guard models_repository_guard; @@ -1059,21 +1059,26 @@ StoragePtr Context::getViewSource() const return view_source; } -void Context::addPlanFragmentViewSource(const String & plan_fragment_id, const StoragePtr & storage) +void Context::addInitialQueryContext(const String & plan_fragment_id, const ContextPtr context) { - std::lock_guard lock(shared->view_sources_mutex); - shared->view_sources[plan_fragment_id] = storage; + std::lock_guard lock(shared->initial_query_contexts_mutex); + shared->initial_query_contexts[plan_fragment_id] = context; } -StoragePtr Context::getPlanFragmentViewSource(const String & plan_fragment_id) const +ContextPtr Context::getInitialQueryContext(const String & plan_fragment_id) const { - std::lock_guard lock(shared->view_sources_mutex); - auto it = shared->view_sources.find(plan_fragment_id); - if (it == shared->view_sources.end()) - throw Exception(ErrorCodes::BAD_GET, "There is no view source for {}", plan_fragment_id); - StoragePtr ret_view_source = it->second; - shared->view_sources.erase(it); - return ret_view_source; + std::lock_guard lock(shared->initial_query_contexts_mutex); + auto it = shared->initial_query_contexts.find(plan_fragment_id); + if (it == shared->initial_query_contexts.end()) + throw Exception(ErrorCodes::BAD_GET, "There is no initial query context for {}", plan_fragment_id); + ContextPtr initial_query_context = it->second; + shared->initial_query_contexts.erase(it); + return initial_query_context; +} + +bool Context::isStandaloneMode() const +{ + return (!getSettingsRef().enable_distributed_plan || (isInitialQuery() && getRunningMode() == Context::RunningMode::STORE)); } Settings Context::getSettings() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 07aed4e2432a..67233dadadc7 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -514,7 +514,7 @@ class Context: public std::enable_shared_from_this const QueryFactoriesInfo & getQueryFactoriesInfo() const { return query_factories_info; } void addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const; - bool isInitialNode() const { return !query_plan_fragment_info; } + bool isInitialQuery() const { return !query_plan_fragment_info; } const QueryPlanFragmentInfo & getQueryPlanFragmentInfo() const { return query_plan_fragment_info.value(); } void setQueryPlanFragmentInfo(const QueryPlanFragmentInfo & query_plan_fragment_info_) { query_plan_fragment_info = query_plan_fragment_info_; } @@ -525,8 +525,10 @@ class Context: public std::enable_shared_from_this void addViewSource(const StoragePtr & storage); StoragePtr getViewSource() const; - void addPlanFragmentViewSource(const String & plan_fragment_id, const StoragePtr & storage); - StoragePtr getPlanFragmentViewSource(const String & plan_fragment_id) const; + + void addInitialQueryContext(const String & plan_fragment_id, const ContextPtr context); + ContextPtr getInitialQueryContext(const String & plan_fragment_id) const; + bool isStandaloneMode() const; String getCurrentDatabase() const; String getCurrentQueryId() const { return client_info.current_query_id; } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 03b7f6a77fcf..33746c1807e7 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -438,7 +438,7 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) "SystemNumbers", "SystemOne", "SystemZeros", "SystemContributors", "SystemLicenses"}; - static std::unordered_set special_storages{"HDFS", "S3", "MySQL", "Input"}; + static std::unordered_set special_storages{"HDFS", "S3", "MySQL"}; auto fillStage = [&store_replicas, &compute_replicas, this, &my_replica](Stage * stage) { @@ -461,6 +461,10 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) { stage->has_view_source = true; } + else if (leaf_node->step->getStepDescription() == "Input") + { + stage->has_input_function = true; + } else { LOG_DEBUG( @@ -618,6 +622,7 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) query_info.set_initial_query_id(initial_query_id); query_info.set_stage_id(stage.id); query_info.set_has_view_source(stage.has_view_source); + query_info.set_has_input_function(stage.has_input_function); for (const auto parent : stage.parents) { @@ -636,12 +641,12 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) query_info.set_node_id(*worker); LOG_DEBUG(log, "Remote plan fragment:\n{}", debugRemotePlanFragment(query_info.query(), *worker, initial_query_id, &stage)); - if (stage.has_view_source) + if (stage.has_view_source || stage.has_input_function) { const String & plan_fragment_id = query_info.initial_query_id() + "/" + toString(query_info.stage_id()) + "/" + query_info.node_id(); - context->addPlanFragmentViewSource(plan_fragment_id, context->getViewSource()); - LOG_DEBUG(log, "Add plan fragment view source."); + context->addInitialQueryContext(plan_fragment_id, context->getQueryContext()); + LOG_DEBUG(log, "Store initial query context for plan fragment {}, because has {}.", plan_fragment_id, (stage.has_view_source ? "view source" : "input function")); } GRPCClient cli(*worker); @@ -853,7 +858,7 @@ bool QueryPlan::buildDistributedPlan(ContextMutablePtr context) return false; } /// Query hits directly on the store worker node. - if (context->isInitialNode() && context->getRunningMode() == Context::RunningMode::STORE) + if (context->isInitialQuery() && context->getRunningMode() == Context::RunningMode::STORE) { LOG_DEBUG(log, "Skip building distributed plan, because initial query hits directly on store worker."); return false; @@ -876,7 +881,7 @@ bool QueryPlan::buildDistributedPlan(ContextMutablePtr context) checkInitialized(); optimize(QueryPlanOptimizationSettings::fromContext(context)); - if (context->isInitialNode()) + if (context->isInitialQuery()) { buildStages(context); scheduleStages(context); diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 57be7849e373..9557237c12e0 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -132,6 +132,7 @@ class QueryPlan std::vector leaf_nodes; /// Store leaf nodes which are from right side to left side. bool is_leaf_stage = false; /// Current stage is a leaf stage if it has any leaf node reading data from storage(not from remote). bool has_view_source = false; /// Current stage reads data to trigger materialized view. + bool has_input_function = false; }; /// Note: do not use vector, otherwise pointers to elements in it will be invalidated when vector increases. diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index bfb4b6a9a03c..82289e3a5492 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1171,15 +1171,6 @@ namespace .parent_sources = std::move(parent_sources), .sinks = sinks }; query_context->setQueryPlanFragmentInfo(std::move(fragment_info)); - - if (query_info.has_view_source()) - { - const String & plan_fragment_id - = fragment_info.initial_query_id + "/" + toString(fragment_info.stage_id) + "/" + fragment_info.node_id; - LOG_DEBUG(log, "Get plan fragment view source for {}", plan_fragment_id); - const auto & view_source = query_context->getPlanFragmentViewSource(plan_fragment_id); - query_context->addViewSource(view_source); - } } /// Prepare settings. @@ -1263,25 +1254,46 @@ namespace createExternalTables(); }); - /// Set callbacks to execute function input(). - query_context->setInputInitializer([this] (ContextPtr context, const StoragePtr & input_storage) + if (call_type == CALL_EXECUTE_PLAN_FRAGMENT) { - if (context != query_context) - throw Exception("Unexpected context in Input initializer", ErrorCodes::LOGICAL_ERROR); - input_function_is_used = true; - initializeBlockInputStream(input_storage->getInMemoryMetadataPtr()->getSampleBlock()); - }); + const String & plan_fragment_id + = query_info.initial_query_id() + "/" + toString(query_info.stage_id()) + "/" + query_info.node_id(); - query_context->setInputBlocksReaderCallback([this](ContextPtr context) -> Block + if (query_info.has_view_source() || query_info.has_input_function()) + { + const auto & initial_query_context = query_context->getInitialQueryContext(plan_fragment_id); + if (query_info.has_view_source()) + { + LOG_DEBUG(log, "Restore view source for plan fragment {}", plan_fragment_id); + query_context->addViewSource(initial_query_context->getViewSource()); + } + else if (query_info.has_input_function()) + { + query_context->setQueryContext(std::const_pointer_cast(initial_query_context)); + } + } + } + else { - if (context != query_context) - throw Exception("Unexpected context in InputBlocksReader", ErrorCodes::LOGICAL_ERROR); + /// Set callbacks to execute function input(). + query_context->setInputInitializer([this](ContextPtr context, const StoragePtr & input_storage) { + if (context != query_context) + throw Exception("Unexpected context in Input initializer", ErrorCodes::LOGICAL_ERROR); + input_function_is_used = true; + initializeBlockInputStream(input_storage->getInMemoryMetadataPtr()->getSampleBlock()); + }); - Block block; - while (!block && pipeline_executor->pull(block)); + query_context->setInputBlocksReaderCallback([this](ContextPtr context) -> Block { + if (context != query_context) + throw Exception("Unexpected context in InputBlocksReader", ErrorCodes::LOGICAL_ERROR); - return block; - }); + Block block; + while (!block && pipeline_executor->pull(block)) + ; + + return block; + }); + } /// Start executing the query. const auto * query_end = end; diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index 87c5084ff51f..bb586779c346 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -124,6 +124,7 @@ message QueryInfo { /// Format is "replica_name:grpc_port" repeated string sinks = 24; bool has_view_source = 25; + bool has_input_function = 26; } enum LogsLevel { diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index 2ed7a77b59de..64c67e6c0297 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -64,8 +64,14 @@ Pipe StorageInput::read( /// It is TCP request if we have callbacks for input(). if (query_context->getInputBlocksReaderCallback()) { - /// Send structure to the client. - query_context->initializeInput(shared_from_this()); + /// There are two cases to initialize input. + /// 1. In standalone mode. + /// 2. In distributed mode and current query is not initial query. + if (context->isStandaloneMode() || !context->isInitialQuery()) + { + /// Send structure to the client. + query_context->initializeInput(shared_from_this()); + } return Pipe(std::make_shared(query_context, metadata_snapshot->getSampleBlock())); } From ab7b70667c2c5127fa26de427714f374f95cb12e Mon Sep 17 00:00:00 2001 From: madianjun Date: Tue, 4 Jan 2022 18:22:53 +0800 Subject: [PATCH 27/84] Only optimize_trivial_count on store workers --- src/Interpreters/Aggregator.h | 8 ++++++-- src/Interpreters/ExpressionAnalyzer.h | 2 ++ src/Interpreters/InterpreterSelectQuery.cpp | 16 ++++++++++++---- src/Processors/QueryPlan/QueryPlan.cpp | 14 +++++++++++--- 4 files changed, 31 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 3c53769e128c..0db7db316f24 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -912,6 +912,8 @@ class Aggregator final bool compile_aggregate_expressions; size_t min_count_to_compile_aggregate_expression; + bool optimize_trivial_count; + Params( const Block & src_header_, const ColumnNumbers & keys_, const AggregateDescriptions & aggregates_, @@ -923,7 +925,8 @@ class Aggregator final size_t min_free_disk_space_, bool compile_aggregate_expressions_, size_t min_count_to_compile_aggregate_expression_, - const Block & intermediate_header_ = {}) + const Block & intermediate_header_ = {}, + bool optimize_trivial_count_ = false) : src_header(src_header_), intermediate_header(intermediate_header_), keys(keys_), aggregates(aggregates_), keys_size(keys.size()), aggregates_size(aggregates.size()), @@ -934,7 +937,8 @@ class Aggregator final tmp_volume(tmp_volume_), max_threads(max_threads_), min_free_disk_space(min_free_disk_space_), compile_aggregate_expressions(compile_aggregate_expressions_), - min_count_to_compile_aggregate_expression(min_count_to_compile_aggregate_expression_) + min_count_to_compile_aggregate_expression(min_count_to_compile_aggregate_expression_), + optimize_trivial_count(optimize_trivial_count_) { } diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index b6bb3c5fad51..9be43ea48cf6 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -219,6 +219,8 @@ struct ExpressionAnalysisResult bool optimize_read_in_order = false; bool optimize_aggregation_in_order = false; bool join_has_delayed_stream = false; + /// Whether trivial count has been optimized. + bool optimize_trivial_count = false; ActionsDAGPtr before_array_join; ArrayJoinActionPtr array_join; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 004c719c486c..2a1a197658cf 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1748,7 +1748,8 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc /// Optimization for trivial query like SELECT count() FROM table. bool optimize_trivial_count = - syntax_analyzer_result->optimize_trivial_count + context->getRunningMode() == Context::RunningMode::STORE + && syntax_analyzer_result->optimize_trivial_count && (settings.max_parallel_replicas <= 1) && storage && storage->getName() != "MaterializedMySQL" @@ -1807,8 +1808,13 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc auto prepared_count = std::make_unique(Pipe(std::move(source)), context); prepared_count->setStepDescription("Optimized trivial count"); query_plan.addStep(std::move(prepared_count)); - from_stage = QueryProcessingStage::WithMergeableState; - analysis_result.first_stage = false; + if (context->isInitialQuery()) + { + /// If initial query is running on store worker, skip first stage. + from_stage = QueryProcessingStage::WithMergeableState; + analysis_result.first_stage = false; + } + analysis_result.optimize_trivial_count = true; return; } } @@ -2071,7 +2077,9 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac settings.max_threads, settings.min_free_disk_space_for_temporary_data, settings.compile_aggregate_expressions, - settings.min_count_to_compile_aggregate_expression); + settings.min_count_to_compile_aggregate_expression, + {}, + analysis_result.optimize_trivial_count); SortDescription group_by_sort_description; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 33746c1807e7..73393302f707 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -794,16 +794,24 @@ void QueryPlan::buildPlanFragment(ContextPtr context) node->step = std::move(step); }; + /// If child is AggregatingStep. if (result.child_aggregating_step) { - auto aggregating_step = std::make_unique(*result.child_aggregating_step); + /// If NOT optimize trivial count, replace AggregatingStep with final=false. + if (!result.child_aggregating_step->getParams().optimize_trivial_count) { + auto aggregating_step = std::make_unique(*result.child_aggregating_step); replaceStep(std::move(aggregating_step), last_node); } + /// If optimize trivial count, remove AggregatingStep. + else + { + LOG_DEBUG(log, "Remove step: {}, stage: {}", result.child_aggregating_step->getName(), stage_id); + last_node = last_node->children[0]; + } } - /// If limit step is pushed down, collect (limit + offset) rows. - if (result.child_limit_step) + else if (result.child_limit_step) result.child_limit_step->resetLimitAndOffset(); root = last_node; From fdb9c89053b265699c12e865bf00ce73f4e5b936 Mon Sep 17 00:00:00 2001 From: madianjun Date: Wed, 5 Jan 2022 15:23:13 +0800 Subject: [PATCH 28/84] Select one result from system.merge_tree_settings --- src/Processors/QueryPlan/QueryPlan.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 73393302f707..a2a86218cca8 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -436,7 +436,8 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) "SystemSettings", "SystemSettingsProfiles", "SystemSettingsProfileElements", "SystemZooKeeper", "SystemNumbers", "SystemOne", "SystemZeros", - "SystemContributors", "SystemLicenses"}; + "SystemContributors", "SystemLicenses", + "SystemReplicatedMergeTreeSettings", "SystemMergeTreeSettings"}; static std::unordered_set special_storages{"HDFS", "S3", "MySQL"}; From 13b449a2b30e01405a1b67fcb32a880b29795866 Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Wed, 5 Jan 2022 08:53:51 +0000 Subject: [PATCH 29/84] Improve CREATE/DROP database executed distributedly --- src/Common/ErrorCodes.cpp | 1 + src/Databases/DatabaseReplicated.cpp | 4 +- src/Databases/DatabaseReplicated.h | 2 +- src/Interpreters/Context.h | 1 + src/Interpreters/DatabaseCatalog.cpp | 29 +++++++++++++ src/Interpreters/DatabaseCatalog.h | 15 +++++++ src/Interpreters/InterpreterCreateQuery.cpp | 46 +++++++++++++++------ src/Interpreters/InterpreterDropQuery.cpp | 32 ++++++++------ 8 files changed, 104 insertions(+), 26 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b2e62d914acb..688fdc6d13a1 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -595,6 +595,7 @@ M(625, IO_SETUP_ERROR) \ M(702, GRPC_READ_ERROR) \ M(703, GRPC_CANCEL_ERROR) \ + M(704, CANNOT_DROP_DATABASE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 319ce269b4f5..1018a468f374 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -821,11 +821,13 @@ void DatabaseReplicated::commitCreateTable(const ASTCreateQuery & query, const S DatabaseAtomic::commitCreateTable(query, table, table_metadata_tmp_path, table_metadata_path, query_context); } -void DatabaseReplicated::commitDatabase(ContextPtr query_context) +void DatabaseReplicated::commitDatabase(ContextPtr query_context, const std::function & add_ops) { auto txn = query_context->getZooKeeperMetadataTransaction(); if (txn && txn->isInitialQuery()) { + if (add_ops) + add_ops(txn); txn->commit(); } } diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 6fd0f378c255..f2e361641e6f 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -37,7 +37,7 @@ class DatabaseReplicated : public DatabaseAtomic const String & table_metadata_tmp_path, const String & table_metadata_path, ContextPtr query_context) override; - void commitDatabase(ContextPtr query_context); + void commitDatabase(ContextPtr query_context, const std::function & add_ops = nullptr); void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & statement, ContextPtr query_context) override; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 67233dadadc7..538d6374b747 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -296,6 +296,7 @@ class Context: public std::enable_shared_from_this public: + DistributedDDLGuardPtr distributed_ddl_guard; struct QueryPlanFragmentInfo { String initial_query_id; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 205d7052d0c7..0baee4c26bb4 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -642,6 +642,11 @@ std::unique_lock DatabaseCatalog::getExclusiveDDLGuardForData return std::unique_lock{db_guard.second}; } +DistributedDDLGuardPtr DatabaseCatalog::getDistributedDDLGuard(const String & database, const String & table) +{ + return std::make_shared(getContext()->getZooKeeper(), database, table); +} + bool DatabaseCatalog::isDictionaryExist(const StorageID & table_id) const { auto storage = tryGetTable(table_id, getContext()); @@ -980,4 +985,28 @@ DDLGuard::~DDLGuard() releaseTableLock(); } +DistributedDDLGuard::DistributedDDLGuard(zkutil::ZooKeeperPtr zookeeper_, const String & database_name_, const String & table_name_): zookeeper(zookeeper_) +{ + try + { + auto path = fs::path(DEFAULT_ZOOKEEPER_METADATA_PATH); + if (table_name_.empty()) + path = path / (database_name_ + "_lock"); + else + path = path / database_name_ / "metadata" / (table_name_ + "_lock"); + hold_path = zookeeper->create(path, "", zkutil::CreateMode::Ephemeral); + created = true; + } + catch (...) + { + + } +} + +DistributedDDLGuard::~DistributedDDLGuard() +{ + if (created) + zookeeper->tryRemove(hold_path); +} + } diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 6079553b0255..a2a71ec36c7d 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -76,7 +77,20 @@ class DDLGuard bool is_database_guard = false; }; +class DistributedDDLGuard +{ +public: + DistributedDDLGuard(zkutil::ZooKeeperPtr zookeeper_, const String & database_name_, const String & table_name_ = ""); + bool isCreated() {return created;} + ~DistributedDDLGuard(); +private: + bool created = false; + String hold_path; + zkutil::ZooKeeperPtr zookeeper = nullptr; +}; + using DDLGuardPtr = std::unique_ptr; +using DistributedDDLGuardPtr = std::shared_ptr; /// Creates temporary table in `_temporary_and_external_tables` with randomly generated unique StorageID. @@ -139,6 +153,7 @@ class DatabaseCatalog : boost::noncopyable, WithMutableContext /// Get an object that protects the database from concurrent DDL queries all tables in the database std::unique_lock getExclusiveDDLGuardForDatabase(const String & database); + DistributedDDLGuardPtr getDistributedDDLGuard(const String & database, const String & table = ""); void assertDatabaseExists(const String & database_name) const; void assertDatabaseDoesntExist(const String & database_name) const; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 60445ef327cf..f0dda070ffc5 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -88,6 +88,7 @@ namespace ErrorCodes extern const int UNKNOWN_DATABASE; extern const int PATH_ACCESS_DENIED; extern const int NOT_IMPLEMENTED; + extern const int CANNOT_CREATE_DATABASE; } namespace fs = std::filesystem; @@ -102,7 +103,14 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) { String database_name = create.database; - auto guard = DatabaseCatalog::instance().getDDLGuard(database_name, ""); + /// If the statement is not from ddl and not internal, that is from the user, the database requires distributed ddl guard + if (!getContext()->getClientInfo().is_replicated_database_internal && !internal) + { + getContext()->distributed_ddl_guard = DatabaseCatalog::instance().getDistributedDDLGuard(database_name); + if (!getContext()->distributed_ddl_guard->isCreated()) + throw Exception( + "Database " + database_name + " is locked by another one, couldn't acquire lock", ErrorCodes::CANNOT_CREATE_DATABASE); + } /// Database can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard if (DatabaseCatalog::instance().isDatabaseExist(database_name)) @@ -241,7 +249,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) { auto * ptr = typeid_cast( DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); - guard.reset(); if (!ptr) throw Exception("The default database is not Replicated engine", ErrorCodes::LOGICAL_ERROR); return ptr->tryEnqueueReplicatedDDL(query_ptr, getContext()); @@ -249,13 +256,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", getContext()); - if (getContext()->getClientInfo().is_replicated_database_internal && !internal) - { - auto * ptr = typeid_cast( - DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); - ptr->commitDatabase(getContext()); - } - if (create.uuid != UUIDHelpers::Nil) create.database = TABLE_WITH_UUID_NAME_PLACEHOLDER; @@ -281,13 +281,35 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) out.close(); } - /// We attach database before loading it's tables, so do not allow concurrent DDL queries - auto db_guard = DatabaseCatalog::instance().getExclusiveDDLGuardForDatabase(database_name); - + auto create_database_nodes_in_zookeeper = [this, &database_name] (ZooKeeperMetadataTransactionPtr txn) + { + auto zookeeper_path = fs::path(DEFAULT_ZOOKEEPER_METADATA_PATH) / database_name; + getContext()->getZooKeeper()->createAncestors(zookeeper_path); + + txn->addOp(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); + txn->addOp(zkutil::makeCreateRequest(zookeeper_path / "log", "", zkutil::CreateMode::Persistent)); + txn->addOp(zkutil::makeCreateRequest(zookeeper_path / "replicas", "", zkutil::CreateMode::Persistent)); + txn->addOp(zkutil::makeCreateRequest(zookeeper_path / "counter", "", zkutil::CreateMode::Persistent)); + /// We create and remove counter/cnt- node to increment sequential number of counter/ node and make log entry numbers start from 1. + /// New replicas are created with log pointer equal to 0 and log pointer is a number of the last executed entry. + /// It means that we cannot have log entry with number 0. + txn->addOp(zkutil::makeCreateRequest(zookeeper_path / "counter/cnt-", "", zkutil::CreateMode::Persistent)); + txn->addOp(zkutil::makeRemoveRequest(zookeeper_path / "counter/cnt-", -1)); + txn->addOp(zkutil::makeCreateRequest(zookeeper_path / "metadata", "", zkutil::CreateMode::Persistent)); + txn->addOp(zkutil::makeCreateRequest(zookeeper_path / "max_log_ptr", "1", zkutil::CreateMode::Persistent)); + txn->addOp(zkutil::makeCreateRequest(zookeeper_path / "logs_to_keep", "1000", zkutil::CreateMode::Persistent)); + }; bool added = false; bool renamed = false; try { + if (getContext()->getClientInfo().is_replicated_database_internal && !internal) + { + auto * ptr = typeid_cast( + DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); + ptr->commitDatabase(getContext(), create_database_nodes_in_zookeeper); + } + /// TODO Attach db only after it was loaded. Now it's not possible because of view dependencies DatabaseCatalog::instance().attachDatabase(database_name, database); added = true; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index ade7ae6bae3a..b009d7046253 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -33,6 +33,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int INCORRECT_QUERY; extern const int TABLE_IS_READ_ONLY; + extern const int CANNOT_DROP_DATABASE; } @@ -304,7 +305,14 @@ BlockIO InterpreterDropQuery::executeToDatabase(const ASTDropQuery & query) BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, DatabasePtr & database, std::vector & uuids_to_wait) { const auto & database_name = query.database; - auto ddl_guard = DatabaseCatalog::instance().getDDLGuard(database_name, ""); + + if (!getContext()->getClientInfo().is_replicated_database_internal) + { + getContext()->distributed_ddl_guard = DatabaseCatalog::instance().getDistributedDDLGuard(database_name); + if (!getContext()->distributed_ddl_guard->isCreated()) + throw Exception( + "Database " + database_name + " is locked by another one, couldn't acquire lock", ErrorCodes::CANNOT_DROP_DATABASE); + } database = tryGetDatabase(database_name, query.if_exists); if (database) @@ -325,19 +333,11 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, { auto * ptr = typeid_cast( DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); - ddl_guard.reset(); if (!ptr) throw Exception("The default database is not Replicated engine", ErrorCodes::LOGICAL_ERROR); return ptr->tryEnqueueReplicatedDDL(query_ptr, getContext()); } - if (getContext()->getClientInfo().is_replicated_database_internal) - { - auto * ptr = typeid_cast( - DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); - ptr->commitDatabase(getContext()); - } - #if USE_MYSQL if (database->getEngineName() == "MaterializedMySQL") stopDatabaseSynchronization(database); @@ -365,6 +365,9 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, iterator->table()->flush(); } + /// Just drop the table locally, so no transaction is needed + auto txn = getContext()->getZooKeeperMetadataTransaction(); + getContext()->resetZooKeeperMetadataTransaction(); for (auto iterator = database->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) { DatabasePtr db; @@ -374,6 +377,8 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, executeToTableImpl(query_for_table, db, table_to_wait); uuids_to_wait.push_back(table_to_wait); } + /// Recover transaction for dropping the database + getContext()->initZooKeeperMetadataTransaction(txn); } if (!drop && query.no_delay) @@ -383,12 +388,15 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, database->waitDetachedTableNotInUse(table_uuid); } - /// Protects from concurrent CREATE TABLE queries - auto db_guard = DatabaseCatalog::instance().getExclusiveDDLGuardForDatabase(database_name); - if (!drop) database->assertCanBeDetached(true); + if (getContext()->getClientInfo().is_replicated_database_internal) + { + auto * ptr = typeid_cast( + DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); + ptr->commitDatabase(getContext()); + } /// DETACH or DROP database itself DatabaseCatalog::instance().detachDatabase(getContext(), database_name, drop, database->shouldBeEmptyOnDetach()); } From ef74935c84602cd1c44527d82d838b2e1dd9848f Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Wed, 29 Dec 2021 10:03:35 +0800 Subject: [PATCH 30/84] Set database_replicated_always_detach_permanently default to true --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 548af37e9f8a..9c8f8f85fb5d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -486,7 +486,7 @@ class IColumn; M(Bool, allow_experimental_database_replicated, true, "Allow to create databases with Replicated engine", 0) \ M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \ M(UInt64, max_distributed_depth, 5, "Maximum distributed query depth", 0) \ - M(Bool, database_replicated_always_detach_permanently, false, "Execute DETACH TABLE as DETACH TABLE PERMANENTLY if database engine is Replicated", 0) \ + M(Bool, database_replicated_always_detach_permanently, true, "Execute DETACH TABLE as DETACH TABLE PERMANENTLY if database engine is Replicated", 0) \ M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result", 0) \ M(UInt64, distributed_ddl_entry_format_version, 1, "Version of DDL entry to write into ZooKeeper", 0) \ M(UInt64, external_storage_max_read_rows, 0, "Limit maximum number of rows when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ From 6a3987c582460de2b934c565d8b26b3aa8ffb78d Mon Sep 17 00:00:00 2001 From: caspian Date: Wed, 5 Jan 2022 13:02:44 +0800 Subject: [PATCH 31/84] Add settings for distributed query plan --- src/Processors/QueryPlan/QueryPlan.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index a2a86218cca8..2db1d73512ea 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -625,6 +625,11 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) query_info.set_has_view_source(stage.has_view_source); query_info.set_has_input_function(stage.has_input_function); + for (const auto setting : context->getSettingsRef().allChanged()) + { + (*query_info.mutable_settings())[setting.getName()] = setting.getValueString(); + } + for (const auto parent : stage.parents) { clickhouse::grpc::MapEntry entry; From 16d1740f16cd4a0091e86346df8cb26ca4fbbdf2 Mon Sep 17 00:00:00 2001 From: caspian Date: Wed, 5 Jan 2022 14:36:45 +0800 Subject: [PATCH 32/84] Reduce thread name --- src/Server/GRPCServer.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 82289e3a5492..63dc97af6900 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1038,7 +1038,7 @@ namespace { if (call_type == CALL_EXECUTE_PLAN_FRAGMENT) { - setThreadName("GRPCServerExecutePlanFragment"); + setThreadName("GRPCExecPlan"); /// Include two steps: /// 1.Store query info. @@ -1053,7 +1053,7 @@ namespace } else if (call_type == CALL_FETCH_PLAN_FRAGMENT_RESULT) { - setThreadName("GRPCServerFetchPlanFragmentResult"); + setThreadName("GRPCFetchResult"); receiveTicket(); loadQueryInfoWrapper(); executeQuery(); @@ -1062,7 +1062,7 @@ namespace } else if (call_type == CALL_CANCEL_PLAN_FRAGMENT) { - setThreadName("GRPCServerCancelPlanFragment"); + setThreadName("GRPCCancelPlan"); receiveTicket(); loadQueryInfoWrapper(true); if (query_info_wrapper) From 9f202c5f926ad13aba5382b3c2bde0b36568d7bf Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Thu, 6 Jan 2022 12:59:49 +0800 Subject: [PATCH 33/84] throw exception --- src/Server/GRPCServer.cpp | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 63dc97af6900..38a436aa3f85 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -806,6 +806,7 @@ namespace std::atomic finish{false}; std::atomic cancel{false}; std::chrono::seconds wait_timeout_seconds{600}; + Exception exception; }; void QueryInfoWrapper::notifyHeader(Block header_) @@ -1767,6 +1768,9 @@ namespace throwIfFailedToSendResult(); } + if (query_info_wrapper->exception.code()) + throw Exception(query_info_wrapper->exception); + if (!query_info_wrapper->cancel) { addTotalsToResult(query_info_wrapper->totals); @@ -1841,6 +1845,18 @@ namespace void Call::onException(const Exception & exception) { + if (call_type == CALL_EXECUTE_PLAN_FRAGMENT) + { + if (query_info_wrapper) + { + LOG_DEBUG(log, "{} producer has an exception", query_info_key); + query_info_wrapper->exception = exception; + cancelPlanFragment(); + query_info_wrapper->notifyFinish(); + } + query_info_key.clear(); + } + io.onException(); LOG_ERROR(log, getExceptionMessage(exception, true)); @@ -1911,7 +1927,7 @@ namespace query_scope.reset(); query_context.reset(); session.reset(); - if (call_type == CALL_EXECUTE_PLAN_FRAGMENT && !query_info_key.empty()) + if (!query_info_key.empty()) query_info_map->erase(query_info_key); } From ec9ee6a6be50e495a6605b7ece4bd60ba4fbaf77 Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Thu, 6 Jan 2022 11:13:31 +0000 Subject: [PATCH 34/84] Sync databases metadata with meta-service, and remove local files for databases not exist in meta-service --- src/Interpreters/DDLTask.cpp | 8 +++++ src/Interpreters/loadMetadata.cpp | 52 ++++++++++++++++++++++++++++--- 2 files changed, 55 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 3d8297c1c1a2..85442e1ec1ba 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -352,6 +353,13 @@ void DatabaseReplicatedTask::parseQueryFromEntry(ContextPtr context) { if (!ddl_query->table.empty()) ddl_query->database = database->getDatabaseName(); + else + { + if (auto * create_query = dynamic_cast(query.get())) + create_query->if_not_exists = true; + else if (auto * drop_query = dynamic_cast(query.get())) + drop_query->if_exists = true; + } } } diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 675f40f6c12f..9166bb3b5575 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -164,15 +164,57 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam if (create_default_db_if_not_exists && !metadata_dir_for_default_db_already_exists) databases.emplace(default_database_name, std::filesystem::path(path) / escapeForFileName(default_database_name)); - /// Load databases from metaService + /// Sync databases with meta-service. { auto zookeeper = context->getZooKeeper(); zookeeper->createAncestors(std::filesystem::path(DEFAULT_ZOOKEEPER_METADATA_PATH) / ""); - auto db_names = zookeeper->getChildren(DEFAULT_ZOOKEEPER_METADATA_PATH); - for (auto & db_name : db_names) + auto get_consistent_metadata_snapshot = [&zookeeper] { + std::map database_name_to_metadata; + auto zookeeper_path = std::filesystem::path(DEFAULT_ZOOKEEPER_METADATA_PATH); + Strings database_names = zookeeper->getChildren(zookeeper_path); + for (const auto & database_name : database_names) + { + String query = zookeeper->get(zookeeper_path / database_name); + database_name_to_metadata.emplace(unescapeForFileName(database_name), query); + } + return database_name_to_metadata; + }; + auto database_name_to_metadata = get_consistent_metadata_snapshot(); + /// Remove local metadata for databases that do not exist in meta-service. + Strings databases_to_detach; + for (const auto & [database_name, _] : databases) + if (!isSystemOrInformationSchema(database_name) && database_name != default_database_name + && database_name_to_metadata.find(database_name) == database_name_to_metadata.end()) + databases_to_detach.emplace_back(database_name); + + for (const auto & database_to_detach : databases_to_detach) + { + databases.erase(database_to_detach); + String escaped_name = escapeForFileName(database_to_detach); + /// Delete meta data + fs::path database_metadata_dir = fs::path(path) / escaped_name; + if (fs::is_symlink(database_metadata_dir)) + fs::remove_all(fs::read_symlink(database_metadata_dir)); + fs::remove_all(database_metadata_dir); + fs::path database_metadata_file = fs::path(path) / (escaped_name + ".sql"); + fs::remove(database_metadata_file); + /// Delete data + fs::path database_data_dir = fs::path(context->getPath()) / "data" / escaped_name; + fs::directory_iterator database_dir_end; + for (fs::directory_iterator it(database_data_dir); it != database_dir_end; ++it) + { + auto table_dir = it->path(); + if (fs::is_symlink(table_dir)) + table_dir = fs::read_symlink(table_dir); + fs::remove_all(table_dir); + } + fs::remove_all(database_data_dir); + } + + for (const auto & [database_name, _] : database_name_to_metadata) { - if (!databases.contains(db_name)) - databases.emplace(db_name, std::filesystem::path(path) / escapeForFileName(db_name)); + if (!databases.contains(database_name)) + databases.emplace(database_name, std::filesystem::path(path) / escapeForFileName(database_name)); } } From c59241d1493797676186c6d0eaa1e543bdbb8b26 Mon Sep 17 00:00:00 2001 From: caspian Date: Fri, 7 Jan 2022 09:27:54 +0000 Subject: [PATCH 35/84] Add persistent recursive watch --- src/Common/ZooKeeper/IKeeper.cpp | 2 + src/Common/ZooKeeper/IKeeper.h | 30 ++ src/Common/ZooKeeper/Types.h | 2 + src/Common/ZooKeeper/ZooKeeper.cpp | 18 ++ src/Common/ZooKeeper/ZooKeeperCommon.cpp | 39 ++- src/Common/ZooKeeper/ZooKeeperCommon.h | 44 +++ src/Common/ZooKeeper/ZooKeeperConstants.cpp | 6 + src/Common/ZooKeeper/ZooKeeperConstants.h | 2 + .../KeeperPersistentWatcherMgr.cpp | 68 +++++ src/Coordination/KeeperPersistentWatcherMgr.h | 78 +++++ src/Coordination/KeeperStorage.cpp | 273 ++++++++++++++---- src/Coordination/KeeperStorage.h | 7 +- src/Interpreters/ZooKeeperLog.cpp | 2 + 13 files changed, 519 insertions(+), 52 deletions(-) create mode 100644 src/Coordination/KeeperPersistentWatcherMgr.cpp create mode 100644 src/Coordination/KeeperPersistentWatcherMgr.h diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index 4f0c5efe680f..3a2efc7bf656 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -144,6 +144,8 @@ void ListRequest::addRootPath(const String & root_path) { Coordination::addRootP void CheckRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } void SetACLRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } void GetACLRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } +void AddWatchRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } +void RemoveWatchesRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } void MultiRequest::addRootPath(const String & root_path) { diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 65ebd8ee615d..f6ed8be1b99b 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -345,6 +345,36 @@ struct MultiResponse : virtual Response } }; +struct AddWatchRequest : virtual Request +{ + String path; + int32_t mode = 0; + + void addRootPath(const String & root_path) override; + String getPath() const override { return path; } + size_t bytesSize() const override { return path.size() + sizeof(mode); } +}; + +struct AddWatchResponse : virtual Response +{ + int32_t errCode = 0; + size_t bytesSize() const override { return sizeof(errCode); } +}; + +struct RemoveWatchesRequest : virtual Request +{ + String path; + int32_t type = 0; + + void addRootPath(const String & root_path) override; + String getPath() const override { return path; } + size_t bytesSize() const override { return path.size() + sizeof(type); } +}; + +struct RemoveWatchesResponse : virtual Response +{ +}; + /// This response may be received only as an element of responses in MultiResponse. struct ErrorResponse : virtual Response { diff --git a/src/Common/ZooKeeper/Types.h b/src/Common/ZooKeeper/Types.h index 35d4188d3e2a..2c89c819863c 100644 --- a/src/Common/ZooKeeper/Types.h +++ b/src/Common/ZooKeeper/Types.h @@ -33,5 +33,7 @@ Coordination::RequestPtr makeCreateRequest(const std::string & path, const std:: Coordination::RequestPtr makeRemoveRequest(const std::string & path, int version); Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version); Coordination::RequestPtr makeCheckRequest(const std::string & path, int version); +Coordination::RequestPtr makeAddWatchRequest(const std::string & path, int mode); +Coordination::RequestPtr makeRemoveWatchesRequest(const std::string & path, int type); } diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index f05a10b8815a..4b817af049ab 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -1133,4 +1133,22 @@ Coordination::RequestPtr makeCheckRequest(const std::string & path, int version) return request; } +// makeAddWatchRequest add persistent watch or persistent & recursive watch on a znode, mode can be: 0: persistent, 1: persistent-recursive +Coordination::RequestPtr makeAddWatchRequest(const std::string & path, int mode) +{ + auto request = std::make_shared(); + request->path = path; + request->mode = mode; + return request; +} + +// makeRemoveWatchesRequest remove watch on a znode, type can be 1: children watch, 2: data watch, 3: any watches +Coordination::RequestPtr makeRemoveWatchesRequest(const std::string & path, int type) +{ + auto request = std::make_shared(); + request->path = path; + request->type = type; + return request; +} + } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index bc8fe0fe365a..9b5f8afec4cc 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -313,6 +313,40 @@ void ZooKeeperErrorResponse::writeImpl(WriteBuffer & out) const Coordination::write(error, out); } +void ZooKeeperAddWatchRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path, out); + Coordination::write(mode, out); +} + +void ZooKeeperAddWatchRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(path, in); + Coordination::read(mode, in); +} + +void ZooKeeperAddWatchResponse::writeImpl(WriteBuffer &out) const +{ + Coordination::write(errCode, out); +} + +void ZooKeeperAddWatchResponse::readImpl(ReadBuffer &in) +{ + Coordination::read(errCode, in); +} + +void ZooKeeperRemoveWatchesRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path, out); + Coordination::write(type, out); +} + +void ZooKeeperRemoveWatchesRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(path, in); + Coordination::read(type, in); +} + ZooKeeperMultiRequest::ZooKeeperMultiRequest(const Requests & generic_requests, const ACLs & default_acls) { /// Convert nested Requests to ZooKeeperRequests. @@ -503,6 +537,8 @@ ZooKeeperResponsePtr ZooKeeperMultiRequest::makeResponse() const { return std::m ZooKeeperResponsePtr ZooKeeperCloseRequest::makeResponse() const { return std::make_shared(); } ZooKeeperResponsePtr ZooKeeperSetACLRequest::makeResponse() const { return std::make_shared(); } ZooKeeperResponsePtr ZooKeeperGetACLRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperAddWatchRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperRemoveWatchesRequest::makeResponse() const { return std::make_shared(); } void ZooKeeperSessionIDRequest::writeImpl(WriteBuffer & out) const { @@ -728,7 +764,8 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory() registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); - registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); } } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index a51ca9a487ec..ad641f3ecd42 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -425,6 +425,50 @@ struct ZooKeeperGetACLResponse final : GetACLResponse, ZooKeeperResponse size_t bytesSize() const override { return GetACLResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } }; +struct ZooKeeperAddWatchRequest final : AddWatchRequest, ZooKeeperRequest +{ + ZooKeeperAddWatchRequest() = default; + explicit ZooKeeperAddWatchRequest(const AddWatchRequest & base) : AddWatchRequest(base) {} + OpNum getOpNum() const override { return OpNum::AddWatch; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return true; } + + size_t bytesSize() const override { return AddWatchRequest::bytesSize() + sizeof(xid); } +}; + +struct ZooKeeperAddWatchResponse final : AddWatchResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer &) override; + void writeImpl(WriteBuffer &) const override; + OpNum getOpNum() const override { return OpNum::AddWatch; } + + size_t bytesSize() const override { return AddWatchResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } +}; + +struct ZooKeeperRemoveWatchesRequest final : RemoveWatchesRequest, ZooKeeperRequest +{ + ZooKeeperRemoveWatchesRequest() = default; + explicit ZooKeeperRemoveWatchesRequest(const RemoveWatchesRequest & base) : RemoveWatchesRequest(base) {} + OpNum getOpNum() const override { return OpNum::RemoveWatches; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return true; } + + size_t bytesSize() const override { return RemoveWatchesRequest::bytesSize() + sizeof(xid); } +}; + +struct ZooKeeperRemoveWatchesResponse final : RemoveWatchesResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer &) override {} + void writeImpl(WriteBuffer &) const override {} + OpNum getOpNum() const override { return OpNum::RemoveWatches; } + + size_t bytesSize() const override { return RemoveWatchesResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } +}; + struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest { OpNum getOpNum() const override { return OpNum::Multi; } diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.cpp b/src/Common/ZooKeeper/ZooKeeperConstants.cpp index 3f480fb6b2b8..db5b799cfe42 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.cpp +++ b/src/Common/ZooKeeper/ZooKeeperConstants.cpp @@ -24,6 +24,8 @@ static const std::unordered_set VALID_OPERATIONS = static_cast(OpNum::SessionID), static_cast(OpNum::SetACL), static_cast(OpNum::GetACL), + static_cast(OpNum::AddWatch), + static_cast(OpNum::RemoveWatches), }; std::string toString(OpNum op_num) @@ -64,6 +66,10 @@ std::string toString(OpNum op_num) return "SetACL"; case OpNum::GetACL: return "GetACL"; + case OpNum::AddWatch: + return "AddWatch"; + case OpNum::RemoveWatches: + return "RemoveWatches"; } int32_t raw_op = static_cast(op_num); throw Exception("Operation " + std::to_string(raw_op) + " is unknown", Error::ZUNIMPLEMENTED); diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h index ed7afd83628b..84d05f41165d 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.h +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -31,7 +31,9 @@ enum class OpNum : int32_t List = 12, Check = 13, Multi = 14, + RemoveWatches = 18, Auth = 100, + AddWatch = 106, SessionID = 997, /// Special internal request }; diff --git a/src/Coordination/KeeperPersistentWatcherMgr.cpp b/src/Coordination/KeeperPersistentWatcherMgr.cpp new file mode 100644 index 000000000000..875f211d8f08 --- /dev/null +++ b/src/Coordination/KeeperPersistentWatcherMgr.cpp @@ -0,0 +1,68 @@ +#include +#include + +namespace DB +{ +KeeperPersistentWatcherMgr::KeeperPersistentWatcherMgr() +{ + recursiveWatcherCount.store(0); +} +KeeperPersistentWatcherMgr::~KeeperPersistentWatcherMgr() +{ + pWatcherModeMap.clear(); + recursiveWatcherCount.store(0); +} + +bool KeeperPersistentWatcherMgr::containsRecursiveWatch() +{ + return std::atomic_load(&(recursiveWatcherCount)) > 0; +} + +void KeeperPersistentWatcherMgr::setWatcherMode(int64_t sessionID, String path, WatcherType watcherType, WatcherMode mode) +{ + if (mode == WatcherMode::Standard) + { + removeWatcher(sessionID, path, watcherType); + } + else + { + WatcherMode oldMode = WatcherMode::Standard; + WatcherKey key(sessionID, path, watcherType); + auto k = pWatcherModeMap.find(key); + if (k != pWatcherModeMap.end()) + oldMode = k->second; + + pWatcherModeMap.insert_or_assign(key, mode); + changeRecursiveCount(oldMode, mode); + } +} + +WatcherMode KeeperPersistentWatcherMgr::getWatcherMode(int64_t sessionID, String path, WatcherType watcherType) +{ + auto k = pWatcherModeMap.find(WatcherKey(sessionID, path, watcherType)); + if (k == pWatcherModeMap.end()) + return WatcherMode::Standard; + return k->second; +} + +void KeeperPersistentWatcherMgr::removeWatcher(int64_t sessionID, String path, WatcherType watcherType) +{ + auto node = pWatcherModeMap.extract(WatcherKey(sessionID, path, watcherType)); + if (!node.empty()) + changeRecursiveCount(node.mapped(), WatcherMode::Standard); +} + +void KeeperPersistentWatcherMgr::changeRecursiveCount(WatcherMode oldMode, WatcherMode newMode) +{ + if (oldMode < WatcherMode::Standard || oldMode > WatcherMode::PersistentRecursive) + oldMode = WatcherMode::Standard; + + if (oldMode != newMode && (oldMode == WatcherMode::PersistentRecursive || newMode == WatcherMode::PersistentRecursive)) + { + if (newMode == WatcherMode::PersistentRecursive) + recursiveWatcherCount.fetch_add(1, std::memory_order_relaxed); + else + recursiveWatcherCount.fetch_sub(1, std::memory_order_relaxed); + } +} +} diff --git a/src/Coordination/KeeperPersistentWatcherMgr.h b/src/Coordination/KeeperPersistentWatcherMgr.h new file mode 100644 index 000000000000..6c3c117453d6 --- /dev/null +++ b/src/Coordination/KeeperPersistentWatcherMgr.h @@ -0,0 +1,78 @@ +#include +#include +#include + +namespace DB +{ +using namespace DB; + +enum struct WatcherType : int8_t +{ + Children = 1, + Data = 2, + Any = 3, +}; + +struct WatcherKey +{ + int64_t sessionID; + String path; + WatcherType watcherType; + WatcherKey(int64_t sessionID_, String path_, WatcherType watcherType_) : sessionID(sessionID_), path(path_), watcherType(watcherType_) { } +}; + +struct WatcherKeyHash +{ + std::size_t operator()(const WatcherKey & key) const + { + return std::hash()(key.path) + std::hash()(key.sessionID) + std::hash()(key.path) + + std::size_t(key.watcherType); + } +}; + +struct WatcherKeyEqual +{ + bool operator()(const WatcherKey & k1, const WatcherKey & k2) const + { + return k1.path == k2.path && k1.path == k2.path && k1.sessionID == k2.sessionID && k1.watcherType == k2.watcherType; + } +}; + +/// WatchMode is the mode of a wather, the old watcher (zokeeper<3.6.0) is Standard, Persistent and PersistentRecursive are added in 3.6.0 +/// If test by using zookeeper client(eg: zkCli.sh), it's necessary to set zookeeper.disableAutoWatchReset=true on zookeeper client config +/// And test it by: ./zkCli.sh -client-configuration ../conf/zkcli.cfg -server localhost:2181 +enum struct WatcherMode : int8_t +{ + Standard = -1, /// not used by the zookeeper raw TCP API, just for inner use + Persistent = 0, + PersistentRecursive = 1, +}; + +/// KeeperPersistentWatcherMgr holds all persistent watchers +/// Because watcher type can be changed, so KeeperPersistentWatcherMgr should also be notified. +class KeeperPersistentWatcherMgr +{ +public: + KeeperPersistentWatcherMgr(); + ~KeeperPersistentWatcherMgr(); + + /// Save a watcher and it's mode, watcherType should only be WatcherType::Children or WatcherType::Data + void setWatcherMode(int64_t sessionID, String path, WatcherType watcherType, WatcherMode mode); + + /// Get a watcher's mode, watcherType should only be WatcherType::Children or WatcherType::Data + WatcherMode getWatcherMode(int64_t sessionID, String path, WatcherType watcherType); + + /// Delete a watcher's mode, watcherType should only be WatcherType::Children or WatcherType::Data + void removeWatcher(int64_t sessionID, String path, WatcherType watcherType); + + /// Whether contains any recursive watcher + bool containsRecursiveWatch(); + +private: + std::unordered_map pWatcherModeMap; + std::atomic recursiveWatcherCount; + + /// Change the recursive watcher count, all mode should be processed (include Standard) + void changeRecursiveCount(WatcherMode oldMode, WatcherMode newMode); +}; +} diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index b6938da958b5..d3b2537e26f5 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -132,11 +132,10 @@ static bool fixupACL( return valid_found; } -static KeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, Coordination::Event event_type) +static void triggerWatches(const String rawPath, const String path, int depth, Coordination::Event event_type, KeeperStorage::Watches & cWatches, WatcherType watcherType, KeeperPersistentWatcherMgr & pWatcherMgr, KeeperStorage::ResponsesForSessions & result) { - KeeperStorage::ResponsesForSessions result; - auto it = watches.find(path); - if (it != watches.end()) + auto sIds_it = cWatches.find(path); + if (sIds_it != cWatches.end()) { std::shared_ptr watch_response = std::make_shared(); watch_response->path = path; @@ -144,46 +143,63 @@ static KeeperStorage::ResponsesForSessions processWatchesImpl(const String & pat watch_response->zxid = -1; watch_response->type = event_type; watch_response->state = Coordination::State::CONNECTED; - for (auto watcher_session : it->second) - result.push_back(KeeperStorage::ResponseForSession{watcher_session, watch_response}); - watches.erase(it); - } + auto & sIds = sIds_it->second; + for (auto watcher_session = sIds.begin(); watcher_session != sIds.end(); ) + { + // reset fields + watch_response->path = path; + watch_response->type = event_type; - auto parent_path = parentPath(path); + auto watchMode = pWatcherMgr.getWatcherMode(*watcher_session, path, watcherType); + bool matched = false; - Strings paths_to_check_for_list_watches; - if (event_type == Coordination::Event::CREATED) - { - paths_to_check_for_list_watches.push_back(parent_path); /// Trigger list watches for parent - } - else if (event_type == Coordination::Event::DELETED) - { - paths_to_check_for_list_watches.push_back(path); /// Trigger both list watches for this path - paths_to_check_for_list_watches.push_back(parent_path); /// And for parent path + if ((watcherType == WatcherType::Data && event_type == Coordination::Event::CHANGED) + || (watcherType == WatcherType::Children && event_type != Coordination::Event::CHANGED)) + { + if (watchMode == WatcherMode::PersistentRecursive) + { + matched = true; + // in persistent-recursive mode, children changed event is redundant, so the node add/delete event path should be the raw path + watch_response->path = rawPath; + } + else if (depth == 0 || (depth == 1 && watcherType == WatcherType::Data)) + matched = true; + } + + if (matched) + { + // in children standard watch, the node add/delete event should be convert to children changed event + if (watchMode != WatcherMode::PersistentRecursive && watcherType == WatcherType::Children && event_type != Coordination::Event::CHANGED) + watch_response->type = Coordination::Event::CHILD; + + result.push_back(KeeperStorage::ResponseForSession{*watcher_session, watch_response}); + } + + if (matched && watchMode == WatcherMode::Standard) + watcher_session = sIds.erase(watcher_session); + else + watcher_session++; + } } - /// CHANGED event never trigger list wathes +} - for (const auto & path_to_check : paths_to_check_for_list_watches) +static KeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, Coordination::Event event_type, KeeperPersistentWatcherMgr & pWatcherMgr) +{ + KeeperStorage::ResponsesForSessions result; + int depth = 0; + for (String curPath = path; !curPath.empty(); curPath = parentPath(curPath)) { - it = list_watches.find(path_to_check); - if (it != list_watches.end()) - { - std::shared_ptr watch_list_response = std::make_shared(); - watch_list_response->path = path_to_check; - watch_list_response->xid = Coordination::WATCH_XID; - watch_list_response->zxid = -1; - if (path_to_check == parent_path) - watch_list_response->type = Coordination::Event::CHILD; - else - watch_list_response->type = Coordination::Event::DELETED; + if (pWatcherMgr.containsRecursiveWatch() || event_type == Coordination::Event::CHANGED) + triggerWatches(path, curPath, depth, event_type, watches, WatcherType::Data, pWatcherMgr, result); - watch_list_response->state = Coordination::State::CONNECTED; - for (auto watcher_session : it->second) - result.push_back(KeeperStorage::ResponseForSession{watcher_session, watch_list_response}); + if (pWatcherMgr.containsRecursiveWatch() || (event_type == Coordination::Event::CREATED || event_type == Coordination::Event::DELETED)) + triggerWatches(path, curPath, depth, event_type, list_watches, WatcherType::Children, pWatcherMgr, result); - list_watches.erase(it); - } + if (curPath == "/" || (depth > 1 && !pWatcherMgr.containsRecursiveWatch())) + break; + + depth++; } return result; } @@ -205,7 +221,7 @@ struct KeeperStorageRequestProcessor : zk_request(zk_request_) {} virtual std::pair process(KeeperStorage & storage, int64_t zxid, int64_t session_id) const = 0; - virtual KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & /*watches*/, KeeperStorage::Watches & /*list_watches*/) const { return {}; } + virtual KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & /*watches*/, KeeperStorage::Watches & /*list_watches*/, KeeperPersistentWatcherMgr & /*pWatcherMgr*/) const { return {}; } virtual bool checkAuth(KeeperStorage & /*storage*/, int64_t /*session_id*/) const { return true; } virtual ~KeeperStorageRequestProcessor() = default; @@ -236,9 +252,9 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr { using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override + KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, KeeperPersistentWatcherMgr & pWatcherMgr) const override { - return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED); + return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED, pWatcherMgr); } bool checkAuth(KeeperStorage & storage, int64_t session_id) const override @@ -534,9 +550,9 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return { response_ptr, undo }; } - KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override + KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, KeeperPersistentWatcherMgr & pWatcherMgr) const override { - return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED); + return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED, pWatcherMgr); } }; @@ -638,9 +654,9 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce return { response_ptr, undo }; } - KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override + KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, KeeperPersistentWatcherMgr & pWatcherMgr) const override { - return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED); + return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED, pWatcherMgr); } }; @@ -931,12 +947,12 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro } } - KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override + KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, KeeperPersistentWatcherMgr & pWatcherMgr) const override { KeeperStorage::ResponsesForSessions result; for (const auto & generic_request : concrete_requests) { - auto responses = generic_request->processWatches(watches, list_watches); + auto responses = generic_request->processWatches(watches, list_watches, pWatcherMgr); result.insert(result.end(), responses.begin(), responses.end()); } return result; @@ -988,6 +1004,121 @@ struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProc } }; +struct KeeperStorageAddWatchRequestProcessor final : public KeeperStorageRequestProcessor +{ + bool checkAuth(KeeperStorage & storage, int64_t session_id) const override + { + auto & container = storage.container; + auto it = container.find(parentPath(zk_request->getPath())); + if (it == container.end()) + return true; + + const auto & node_acls = storage.acl_map.convertNumber(it->value.acl_id); + if (node_acls.empty()) + return true; + + const auto & session_auths = storage.session_and_auth[session_id]; + return checkACL(Coordination::ACL::Delete, node_acls, session_auths); + } + + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + std::pair process(KeeperStorage & storage, int64_t /*zxid*/, int64_t session_id) const override + { + auto & container = storage.container; + Coordination::ZooKeeperAddWatchRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperAddWatchResponse & response = dynamic_cast(*response_ptr); + + auto it = container.find(request.getPath()); + if (it == container.end()) + { + response.error = Coordination::Error::ZNONODE; + } + else + { + auto path_prefix = request.getPath(); + if (path_prefix.empty()) + throw DB::Exception("Logical error: path cannot be empty", ErrorCodes::LOGICAL_ERROR); + + if (request.mode < int32_t(WatcherMode::Persistent) || request.mode > int32_t(WatcherMode::PersistentRecursive)) + throw DB::Exception("Logical error: watch mode is invalid", ErrorCodes::LOGICAL_ERROR); + + storage.list_watches[request.getPath()].insert(session_id); + storage.pWatcherMgr.setWatcherMode(session_id, request.getPath(), WatcherType::Children, WatcherMode(request.mode)); + + storage.watches[request.getPath()].insert(session_id); + storage.pWatcherMgr.setWatcherMode(session_id, request.getPath(), WatcherType::Data, WatcherMode(request.mode)); + + storage.sessions_and_watchers[session_id].emplace(request.getPath()); + + response.error = Coordination::Error::ZOK; + } + + return { response_ptr, {} }; + } +}; + +struct KeeperStorageRemoveWatchesRequestProcessor final : public KeeperStorageRequestProcessor +{ + bool checkAuth(KeeperStorage & storage, int64_t session_id) const override + { + auto & container = storage.container; + auto it = container.find(parentPath(zk_request->getPath())); + if (it == container.end()) + return true; + + const auto & node_acls = storage.acl_map.convertNumber(it->value.acl_id); + if (node_acls.empty()) + return true; + + const auto & session_auths = storage.session_and_auth[session_id]; + return checkACL(Coordination::ACL::Delete, node_acls, session_auths); + } + + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + std::pair process(KeeperStorage & storage, int64_t /*zxid*/, int64_t session_id) const override + { + auto & container = storage.container; + Coordination::ZooKeeperRemoveWatchesRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperRemoveWatchesResponse & response = dynamic_cast(*response_ptr); + + auto it = container.find(request.path); + if (it == container.end()) + { + response.error = Coordination::Error::ZNONODE; + } + else + { + auto path_prefix = request.path; + if (path_prefix.empty()) + throw DB::Exception("Logical error: path cannot be empty", ErrorCodes::LOGICAL_ERROR); + + switch (request.type) + { + case int32_t(WatcherType::Children): + storage.deleteWatcher(storage.list_watches, session_id, request.getPath()); + storage.pWatcherMgr.removeWatcher(session_id, request.path, WatcherType::Children); + break; + case int32_t(WatcherType::Data): + storage.deleteWatcher(storage.watches, session_id, request.getPath()); + storage.pWatcherMgr.removeWatcher(session_id, request.path, WatcherType::Data); + break; + case int32_t(WatcherType::Any): + storage.deleteWatcher(storage.list_watches, session_id, request.getPath()); + storage.pWatcherMgr.removeWatcher(session_id, request.path, WatcherType::Children); + storage.deleteWatcher(storage.watches, session_id, request.getPath()); + storage.pWatcherMgr.removeWatcher(session_id, request.path, WatcherType::Data); + break; + default: + throw DB::Exception("Logical error: watch type is invalid", ErrorCodes::LOGICAL_ERROR); + } + } + + return { response_ptr, {} }; + } +}; + void KeeperStorage::finalize() { if (finalized) @@ -1065,6 +1196,8 @@ KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory() registerKeeperRequestProcessor(*this); registerKeeperRequestProcessor(*this); registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor(*this); } @@ -1096,7 +1229,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(const Coordina parent.children.erase(getBaseName(ephemeral_path)); }); - auto responses = processWatchesImpl(ephemeral_path, watches, list_watches, Coordination::Event::DELETED); + auto responses = processWatchesImpl(ephemeral_path, watches, list_watches, Coordination::Event::DELETED, pWatcherMgr); results.insert(results.end(), responses.begin(), responses.end()); } ephemerals.erase(it); @@ -1144,24 +1277,34 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(const Coordina { if (response->error == Coordination::Error::ZOK) { - auto & watches_type = zk_request->getOpNum() == Coordination::OpNum::List || zk_request->getOpNum() == Coordination::OpNum::SimpleList + WatcherType wt; + if (zk_request->getOpNum() == Coordination::OpNum::List || zk_request->getOpNum() == Coordination::OpNum::SimpleList) + wt = WatcherType::Children; + else + wt = WatcherType::Data; + + auto & watches_type = (wt == WatcherType::Children) ? list_watches : watches; - watches_type[zk_request->getPath()].emplace_back(session_id); + watches_type[zk_request->getPath()].insert(session_id); sessions_and_watchers[session_id].emplace(zk_request->getPath()); + + pWatcherMgr.setWatcherMode(session_id, zk_request->getPath(), wt, WatcherMode::Standard); } else if (response->error == Coordination::Error::ZNONODE && zk_request->getOpNum() == Coordination::OpNum::Exists) { - watches[zk_request->getPath()].emplace_back(session_id); + watches[zk_request->getPath()].insert(session_id); sessions_and_watchers[session_id].emplace(zk_request->getPath()); + + pWatcherMgr.setWatcherMode(session_id, zk_request->getPath(), WatcherType::Data, WatcherMode::Standard); } } /// If this requests processed successfully we need to check watches if (response->error == Coordination::Error::ZOK) { - auto watch_responses = request_processor->processWatches(watches, list_watches); + auto watch_responses = request_processor->processWatches(watches, list_watches, pWatcherMgr); results.insert(results.end(), watch_responses.begin(), watch_responses.end()); } @@ -1214,10 +1357,40 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) if (list_watches_for_path.empty()) list_watches.erase(list_watch); } + + pWatcherMgr.removeWatcher(session_id, watch_path, WatcherType::Children); + pWatcherMgr.removeWatcher(session_id, watch_path, WatcherType::Data); } sessions_and_watchers.erase(watches_it); } } +bool KeeperStorage::deleteWatcher(Watches & cWatches, int64_t sessionID, String path) +{ + bool ret = false; + auto path_it = sessions_and_watchers.find(sessionID); + if (path_it != sessions_and_watchers.end()) + { + auto sIds_it = cWatches.find(path); + if (sIds_it != cWatches.end()) + { + auto & sIds = sIds_it->second; + for (auto s_it = sIds.begin(); s_it != sIds.end(); ) + { + if (*s_it == sessionID) + { + s_it = sIds.erase(s_it); + ret = true; + } else { + s_it++; + } + } + if (sIds.empty()) + cWatches.erase(sIds_it); + } + } + return ret; +} } + diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index bc9a81bc4841..f9ec01ae4b64 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -71,7 +72,7 @@ class KeeperStorage using Container = SnapshotableHashTable; using Ephemerals = std::unordered_map>; using SessionAndWatcher = std::unordered_map>; - using SessionIDs = std::vector; + using SessionIDs = std::unordered_set; /// Just vector of SHA1 from user:password using AuthIDs = std::vector; @@ -104,6 +105,7 @@ class KeeperStorage /// Currently active watches (node_path -> subscribed sessions) Watches watches; Watches list_watches; /// Watches for 'list' request (watches on children). + KeeperPersistentWatcherMgr pWatcherMgr; void clearDeadWatches(int64_t session_id); @@ -176,6 +178,9 @@ class KeeperStorage { return session_expiry_queue.getExpiredSessions(); } + + /// Delete watcher in cWatches by it's sessionID and path + bool deleteWatcher(Watches & cWatches, int64_t sessionID, String path); }; using KeeperStoragePtr = std::unique_ptr; diff --git a/src/Interpreters/ZooKeeperLog.cpp b/src/Interpreters/ZooKeeperLog.cpp index fdcbe4308340..3d431fd4643d 100644 --- a/src/Interpreters/ZooKeeperLog.cpp +++ b/src/Interpreters/ZooKeeperLog.cpp @@ -85,6 +85,8 @@ NamesAndTypesList ZooKeeperLogElement::getNamesAndTypes() {"Multi", static_cast(Coordination::OpNum::Multi)}, {"Auth", static_cast(Coordination::OpNum::Auth)}, {"SessionID", static_cast(Coordination::OpNum::SessionID)}, + {"AddWatch", static_cast(Coordination::OpNum::AddWatch)}, + {"RemoveWatches", static_cast(Coordination::OpNum::RemoveWatches)}, }); auto error_enum = getCoordinationErrorCodesEnumType(); From bb69892f5d764ffa0e271edbec0f95a4307a002d Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Mon, 10 Jan 2022 16:59:58 +0800 Subject: [PATCH 36/84] Fix dangling pointer of query_info in GRPCServer --- src/Server/GRPCServer.cpp | 111 +++++++++++++++++++------------------- 1 file changed, 56 insertions(+), 55 deletions(-) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 38a436aa3f85..87e3bf0dbe91 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -760,7 +760,7 @@ namespace CANCEL }; - QueryInfoWrapper(GRPCQueryInfo * query_info_, int consumers_) + QueryInfoWrapper(const std::shared_ptr & query_info_, int consumers_) : query_info(query_info_), consumers(consumers_), blocks(consumers), ready(consumers, false), ready_count(0) { } @@ -780,7 +780,7 @@ namespace QueryInfoWrapper::Status waitReadyOrFinish(int index); void notifyProduce(); - GRPCQueryInfo * query_info; + std::shared_ptr query_info; int consumers; /// Transferred data between producer and consumer. @@ -961,7 +961,7 @@ namespace Progress progress; InternalTextLogsQueuePtr logs_queue; - GRPCQueryInfo query_info; /// We reuse the same messages multiple times. + std::shared_ptr query_info; /// We reuse the same messages multiple times. GRPCResult result; GRPCTicket ticket; @@ -993,7 +993,7 @@ namespace BoolState reading_ticket{false}; std::atomic failed_to_read_query_info = false; std::atomic failed_to_read_ticket = false; - GRPCQueryInfo next_query_info_while_reading; + std::shared_ptr next_query_info_while_reading; GRPCTicket next_ticket_while_reading; std::atomic want_to_cancel = false; std::atomic check_query_info_contains_cancel_only = false; @@ -1100,10 +1100,10 @@ namespace readQueryInfo(); - if (query_info.cancel()) + if (query_info->cancel()) throw Exception("Initial query info cannot set the 'cancel' field", ErrorCodes::INVALID_GRPC_QUERY_INFO); - LOG_DEBUG(log, "Received initial QueryInfo: {}", getQueryDescription(query_info)); + LOG_DEBUG(log, "Received initial QueryInfo: {}", getQueryDescription(*query_info)); } void Call::receiveTicket() @@ -1120,12 +1120,12 @@ namespace /// If this is from fetchPlanFragmentResult(), restore query_info from wrapper, /// and initialize query_context, but don't build pipeline. if (call_type == CALL_FETCH_PLAN_FRAGMENT_RESULT) - query_info = *query_info_wrapper->query_info; + query_info = query_info_wrapper->query_info; /// Retrieve user credentials. - std::string user = query_info.user_name(); - std::string password = query_info.password(); - std::string quota_key = query_info.quota(); + std::string user = query_info->user_name(); + std::string password = query_info->password(); + std::string quota_key = query_info->quota(); Poco::Net::SocketAddress user_address = responder->getClientAddress(); if (user.empty()) @@ -1141,10 +1141,10 @@ namespace /// The user could specify session identifier and session timeout. /// It allows to modify settings, create temporary tables and reuse them in subsequent requests. - if (!query_info.session_id().empty()) + if (!query_info->session_id().empty()) { session->makeSessionContext( - query_info.session_id(), getSessionTimeout(query_info, iserver.config()), query_info.session_check()); + query_info->session_id(), getSessionTimeout(*query_info, iserver.config()), query_info->session_check()); } query_context = session->makeQueryContext(); @@ -1153,7 +1153,7 @@ namespace if (call_type == CALL_EXECUTE_PLAN_FRAGMENT) { std::unordered_map>> parent_sources; - for (const auto & parent : query_info.parent_sources()) + for (const auto & parent : query_info->parent_sources()) { std::vector> sources; for (const auto & source : parent.second.sources()) @@ -1161,14 +1161,14 @@ namespace parent_sources[parent.first] = std::move(sources); } std::vector> sinks; - for (const auto & sink : query_info.sinks()) + for (const auto & sink : query_info->sinks()) { sinks.emplace_back(std::make_shared(sink)); } Context::QueryPlanFragmentInfo fragment_info{ - .initial_query_id = query_info.initial_query_id(), - .stage_id = query_info.stage_id(), - .node_id = query_info.node_id(), + .initial_query_id = query_info->initial_query_id(), + .stage_id = query_info->stage_id(), + .node_id = query_info->node_id(), .parent_sources = std::move(parent_sources), .sinks = sinks }; query_context->setQueryPlanFragmentInfo(std::move(fragment_info)); @@ -1176,16 +1176,16 @@ namespace /// Prepare settings. SettingsChanges settings_changes; - for (const auto & [key, value] : query_info.settings()) + for (const auto & [key, value] : query_info->settings()) { settings_changes.push_back({key, value}); } query_context->checkSettingsConstraints(settings_changes); query_context->applySettingsChanges(settings_changes); - query_context->getClientInfo().query_kind = ClientInfo::QueryKind(query_info.query_kind()); - query_context->getClientInfo().initial_query_id = query_info.initial_query_id(); - query_context->setCurrentQueryId(query_info.query_id()); + query_context->getClientInfo().query_kind = ClientInfo::QueryKind(query_info->query_kind()); + query_context->getClientInfo().initial_query_id = query_info->initial_query_id(); + query_context->setCurrentQueryId(query_info->query_id()); query_scope.emplace(query_context); /// Prepare for sending exceptions and logs. @@ -1201,12 +1201,12 @@ namespace } /// Set the current database if specified. - if (!query_info.database().empty()) - query_context->setCurrentDatabase(query_info.database()); + if (!query_info->database().empty()) + query_context->setCurrentDatabase(query_info->database()); /// Apply compression settings for this call. - if (query_info.has_result_compression()) - responder->setResultCompression(query_info.result_compression()); + if (query_info->has_result_compression()) + responder->setResultCompression(query_info->result_compression()); /// The interactive delay will be used to show progress. interactive_delay = settings.interactive_delay; @@ -1220,7 +1220,7 @@ namespace } /// Parse the query. - query_text = std::move(*(query_info.mutable_query())); + query_text = std::move(*(query_info->mutable_query())); const char * begin = query_text.data(); const char * end = begin + query_text.size(); ParserQuery parser(end); @@ -1235,10 +1235,10 @@ namespace input_format = "Values"; } - input_data_delimiter = query_info.input_data_delimiter(); + input_data_delimiter = query_info->input_data_delimiter(); /// Choose output format. - query_context->setDefaultFormat(query_info.output_format()); + query_context->setDefaultFormat(query_info->output_format()); if (const auto * ast_query_with_output = dynamic_cast(ast.get()); ast_query_with_output && ast_query_with_output->format) { @@ -1258,17 +1258,17 @@ namespace if (call_type == CALL_EXECUTE_PLAN_FRAGMENT) { const String & plan_fragment_id - = query_info.initial_query_id() + "/" + toString(query_info.stage_id()) + "/" + query_info.node_id(); + = query_info->initial_query_id() + "/" + toString(query_info->stage_id()) + "/" + query_info->node_id(); - if (query_info.has_view_source() || query_info.has_input_function()) + if (query_info->has_view_source() || query_info->has_input_function()) { const auto & initial_query_context = query_context->getInitialQueryContext(plan_fragment_id); - if (query_info.has_view_source()) + if (query_info->has_view_source()) { LOG_DEBUG(log, "Restore view source for plan fragment {}", plan_fragment_id); query_context->addViewSource(initial_query_context->getViewSource()); } - else if (query_info.has_input_function()) + else if (query_info->has_input_function()) { query_context->setQueryContext(std::const_pointer_cast(initial_query_context)); } @@ -1308,8 +1308,8 @@ namespace void Call::storeQueryInfoWrapper() { - query_info_key = query_info.initial_query_id() + "/" + toString(query_info.stage_id()); - auto res = query_info_map->insert(query_info_key, std::make_shared(&query_info, query_info.sinks_size())); + query_info_key = query_info->initial_query_id() + "/" + toString(query_info->stage_id()); + auto res = query_info_map->insert(query_info_key, std::make_shared(query_info, query_info->sinks_size())); if (!res.second) { throw Exception("Query info key " + query_info_key + " already exists", ErrorCodes::LOGICAL_ERROR); @@ -1339,7 +1339,7 @@ namespace return; bool has_data_to_insert = (insert_query && insert_query->data) - || !query_info.input_data().empty() || query_info.next_query_info(); + || !query_info->input_data().empty() || query_info->next_query_info(); if (!has_data_to_insert) { if (!insert_query) @@ -1384,30 +1384,30 @@ namespace { if (need_input_data_from_query_info) { - if (need_input_data_delimiter && !query_info.input_data().empty()) + if (need_input_data_delimiter && !query_info->input_data().empty()) { need_input_data_delimiter = false; return {input_data_delimiter.data(), input_data_delimiter.size()}; } need_input_data_from_query_info = false; - if (!query_info.input_data().empty()) + if (!query_info->input_data().empty()) { need_input_data_delimiter = !input_data_delimiter.empty(); - return {query_info.input_data().data(), query_info.input_data().size()}; + return {query_info->input_data().data(), query_info->input_data().size()}; } } - if (!query_info.next_query_info()) + if (!query_info->next_query_info()) break; if (!isInputStreaming(call_type)) throw Exception("next_query_info is allowed to be set only for streaming input", ErrorCodes::INVALID_GRPC_QUERY_INFO); readQueryInfo(); - if (!query_info.query().empty() || !query_info.query_id().empty() || !query_info.settings().empty() - || !query_info.database().empty() || !query_info.input_data_delimiter().empty() || !query_info.output_format().empty() - || query_info.external_tables_size() || !query_info.user_name().empty() || !query_info.password().empty() - || !query_info.quota().empty() || !query_info.session_id().empty()) + if (!query_info->query().empty() || !query_info->query_id().empty() || !query_info->settings().empty() + || !query_info->database().empty() || !query_info->input_data_delimiter().empty() || !query_info->output_format().empty() + || query_info->external_tables_size() || !query_info->user_name().empty() || !query_info->password().empty() + || !query_info->quota().empty() || !query_info->session_id().empty()) { throw Exception("Extra query infos can be used only to add more input data. " "Only the following fields can be set: input_data, next_query_info, cancel", @@ -1417,7 +1417,7 @@ namespace if (isQueryCancelled()) break; - LOG_DEBUG(log, "Received extra QueryInfo: input_data: {} bytes", query_info.input_data().size()); + LOG_DEBUG(log, "Received extra QueryInfo: input_data: {} bytes", query_info->input_data().size()); need_input_data_from_query_info = true; } @@ -1459,7 +1459,7 @@ namespace { while (true) { - for (const auto & external_table : query_info.external_tables()) + for (const auto & external_table : query_info->external_tables()) { String name = external_table.name(); if (name.empty()) @@ -1529,24 +1529,24 @@ namespace } } - if (!query_info.input_data().empty()) + if (!query_info->input_data().empty()) { /// External tables must be created before executing query, /// so all external tables must be send no later sending any input data. break; } - if (!query_info.next_query_info()) + if (!query_info->next_query_info()) break; if (!isInputStreaming(call_type)) throw Exception("next_query_info is allowed to be set only for streaming input", ErrorCodes::INVALID_GRPC_QUERY_INFO); readQueryInfo(); - if (!query_info.query().empty() || !query_info.query_id().empty() || !query_info.settings().empty() - || !query_info.database().empty() || !query_info.input_data_delimiter().empty() - || !query_info.output_format().empty() || !query_info.user_name().empty() || !query_info.password().empty() - || !query_info.quota().empty() || !query_info.session_id().empty()) + if (!query_info->query().empty() || !query_info->query_id().empty() || !query_info->settings().empty() + || !query_info->database().empty() || !query_info->input_data_delimiter().empty() + || !query_info->output_format().empty() || !query_info->user_name().empty() || !query_info->password().empty() + || !query_info->quota().empty() || !query_info->session_id().empty()) { throw Exception("Extra query infos can be used only to add more data to input or more external tables. " "Only the following fields can be set: input_data, external_tables, next_query_info, cancel", @@ -1554,7 +1554,7 @@ namespace } if (isQueryCancelled()) break; - LOG_DEBUG(log, "Received extra QueryInfo: external tables: {}", query_info.external_tables_size()); + LOG_DEBUG(log, "Received extra QueryInfo: external tables: {}", query_info->external_tables_size()); } } @@ -1936,12 +1936,13 @@ namespace auto start_reading = [&] { reading_query_info.set(true); - responder->read(next_query_info_while_reading, [this](bool ok) + next_query_info_while_reading = std::make_shared(); + responder->read(*next_query_info_while_reading, [this](bool ok) { /// Called on queue_thread. if (ok) { - const auto & nqi = next_query_info_while_reading; + const auto & nqi = *next_query_info_while_reading; if (check_query_info_contains_cancel_only) { if (!nqi.query().empty() || !nqi.query_id().empty() || !nqi.settings().empty() || !nqi.database().empty() @@ -1973,7 +1974,7 @@ namespace waited_for_client_writing += client_writing_watch.elapsedNanoseconds(); } throwIfFailedToReadQueryInfo(); - query_info = std::move(next_query_info_while_reading); + query_info = next_query_info_while_reading; initial_query_info_read = true; }; From e20f07ffc15be2a14515cb29dfcbb62c47767544 Mon Sep 17 00:00:00 2001 From: madianjun Date: Tue, 11 Jan 2022 18:36:47 +0800 Subject: [PATCH 37/84] Fix log message in GRPCClient --- src/Client/GRPCClient.cpp | 2 +- src/Processors/Sources/DistributedSource.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/GRPCClient.cpp b/src/Client/GRPCClient.cpp index ccd69dc0a114..914e2801e25a 100644 --- a/src/Client/GRPCClient.cpp +++ b/src/Client/GRPCClient.cpp @@ -96,7 +96,7 @@ Block GRPCClient::read() } throw Exception( - "Read from grpc server " + addr + "failed, " + toString(result.exception().code()) + ", " + result.exception().display_text(), + "Read from grpc server " + addr + " failed, " + toString(result.exception().code()) + ", " + result.exception().display_text(), ErrorCodes::GRPC_READ_ERROR, true); } diff --git a/src/Processors/Sources/DistributedSource.h b/src/Processors/Sources/DistributedSource.h index dbfbb5e97611..091da9679b31 100644 --- a/src/Processors/Sources/DistributedSource.h +++ b/src/Processors/Sources/DistributedSource.h @@ -16,7 +16,7 @@ class DistributedSource : public SourceWithProgress ~DistributedSource() override; Status prepare() override; - String getName() const override { return "Distributed"; } + String getName() const override { return "DistributedSource"; } void onUpdatePorts() override; From 0c3cb9885be8b5780452bf9ccafbdf7596a8ae2f Mon Sep 17 00:00:00 2001 From: madianjun Date: Wed, 5 Jan 2022 18:59:55 +0800 Subject: [PATCH 38/84] Support IN with subquery --- src/Interpreters/ClientInfo.h | 3 + src/Interpreters/Context.h | 4 + .../IInterpreterUnionOrSelectQuery.h | 5 + src/Interpreters/InterpreterSelectQuery.cpp | 14 +- .../InterpreterSelectWithUnionQuery.cpp | 27 +++ .../InterpreterSelectWithUnionQuery.h | 1 + src/Processors/QueryPlan/QueryPlan.cpp | 197 ++++++++++++++++-- src/Processors/QueryPlan/QueryPlan.h | 5 +- src/Storages/IStorage.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.h | 2 + src/Storages/MergeTree/MutateTask.cpp | 19 +- 11 files changed, 245 insertions(+), 34 deletions(-) diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index ad93e09ca183..2446b8eaba01 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -63,6 +63,9 @@ class ClientInfo time_t initial_query_start_time{}; Decimal64 initial_query_start_time_microseconds{}; + /// For executing distributed plan. + String distributed_query; + // OpenTelemetry trace context we received from client, or which we are going // to send to server. OpenTelemetryTraceContext client_trace_context; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 538d6374b747..ab58d598da0b 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -310,6 +310,7 @@ class Context: public std::enable_shared_from_this OpenTelemetryTraceContext query_trace_context; private: + bool skip_distributed_plan = false; std::optional query_plan_fragment_info; /// It has no value if current node is initial compute node. String select_query; @@ -515,6 +516,9 @@ class Context: public std::enable_shared_from_this const QueryFactoriesInfo & getQueryFactoriesInfo() const { return query_factories_info; } void addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const; + bool getSkipDistributedPlan() const { return skip_distributed_plan; } + void setSkipDistributedPlan(bool skip) { skip_distributed_plan = skip; } + bool isInitialQuery() const { return !query_plan_fragment_info; } const QueryPlanFragmentInfo & getQueryPlanFragmentInfo() const { return query_plan_fragment_info.value(); } void setQueryPlanFragmentInfo(const QueryPlanFragmentInfo & query_plan_fragment_info_) { query_plan_fragment_info = query_plan_fragment_info_; } diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index db9cc086e35b..3cd63788ca4c 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB @@ -25,6 +26,8 @@ class IInterpreterUnionOrSelectQuery : public IInterpreter context->addLocalScalar( "_shard_count", Block{{DataTypeUInt32().createColumnConst(1, *options.shard_count), std::make_shared(), "_shard_count"}}); + /// Set distributed_query for sending query info in building distributed plan. + context->getClientInfo().distributed_query = queryToString(query_ptr); } virtual void buildQueryPlan(QueryPlan & query_plan) = 0; @@ -40,6 +43,8 @@ class IInterpreterUnionOrSelectQuery : public IInterpreter void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const override; + const ContextMutablePtr & getContext() const { return context; } + protected: ASTPtr query_ptr; ContextMutablePtr context; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 2a1a197658cf..b8e35a6eb96a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -438,6 +439,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( std::move(subquery_for_sets), std::move(prepared_sets)); + /// Process SQL just like "SELECT ... FROM _temporary_and_external_tables.`_tmp_fbe82e3a-1815-4563-bbe8-2e3a1815e563`" + if (!query_analyzer->getExternalTables().empty()) + context->setSkipDistributedPlan(true); + if (!options.only_analyze) { if (query.sampleSize() && (input_pipe || !storage || !storage->supportsSampling())) @@ -456,7 +461,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Save the new temporary tables in the query context for (const auto & it : query_analyzer->getExternalTables()) if (!context->tryResolveStorageID({"", it.first}, Context::ResolveExternal)) + { context->addExternalTable(it.first, std::move(*it.second)); + LOG_DEBUG(log, "Add external table to context {}", static_cast(context.get())); + } } if (!options.only_analyze || options.modify_inplace) @@ -499,6 +507,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Calculate structure of the result. result_header = getSampleBlockImpl(); + /// Maybe subquery has been rewritten with "_subqueryX", so reset distributed_query. + String maybe_rewritten_query = queryToString(query_ptr); + LOG_DEBUG(log, "[{}] Rewrite \"{}\" to \"{}\"", static_cast(context.get()), context->getClientInfo().distributed_query, maybe_rewritten_query); + context->getClientInfo().distributed_query = std::move(maybe_rewritten_query); }; analyze(shouldMoveToPrewhere()); @@ -1515,7 +1527,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan( auto read_from_pipe = std::make_unique(std::move(pipe)); read_from_pipe->setStepDescription("Read from NullSource"); - query_plan.addStep(std::move(read_from_pipe)); + query_plan.addStep(std::move(read_from_pipe), context_); if (query_info.projection) { diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 72f70b2a9c42..897d4bad2995 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -34,6 +34,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( const ASTPtr & query_ptr_, ContextPtr context_, const SelectQueryOptions & options_, const Names & required_result_column_names) : IInterpreterUnionOrSelectQuery(query_ptr_, context_, options_) + , log(&Poco::Logger::get("InterpreterSelectWithUnionQuery")) { ASTSelectWithUnionQuery * ast = query_ptr->as(); bool require_full_header = ast->hasNonDefaultUnionMode(); @@ -166,6 +167,32 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( options.ignore_limits |= all_nested_ignore_limits; options.ignore_quota |= all_nested_ignore_quota; + context->getClientInfo().distributed_query.clear(); + for (size_t query_num = 0; query_num < num_children; ++query_num) + { + context->getClientInfo().distributed_query += nested_interpreters[query_num]->getContext()->getClientInfo().distributed_query; + + if (query_num < num_children - 1) + { + if (ast->union_mode == ASTSelectWithUnionQuery::Mode::Unspecified) + { + context->getClientInfo().distributed_query += " UNION "; + } + else if (ast->union_mode == ASTSelectWithUnionQuery::Mode::ALL) + { + context->getClientInfo().distributed_query += " UNION ALL "; + } + else if (ast->union_mode == ASTSelectWithUnionQuery::Mode::DISTINCT) + { + context->getClientInfo().distributed_query += " UNION DISTINCT "; + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "When rewriting SELECT with UNION, unimplemented UNION type: {}", ast->union_mode); + } + } + } + LOG_DEBUG(log, "[{}] Rewrite to: {}", static_cast(context.get()), context->getClientInfo().distributed_query); } Block InterpreterSelectWithUnionQuery::getCommonHeaderForUnion(const Blocks & headers) diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index 720632e7be5a..bd1f1b6dd2ab 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -41,6 +41,7 @@ class InterpreterSelectWithUnionQuery : public IInterpreterUnionOrSelectQuery private: std::vector> nested_interpreters; + Poco::Logger * log; static Block getCommonHeaderForUnion(const Blocks & headers); diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 2db1d73512ea..7a88ad54c517 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -19,6 +19,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -63,7 +66,7 @@ const DataStream & QueryPlan::getCurrentDataStream() const return root->step->getOutputStream(); } -void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vector> plans) +void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vector> plans, ContextPtr context) { if (isInitialized()) throw Exception("Cannot unite plans because current QueryPlan is already initialized", @@ -93,7 +96,7 @@ void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vectornodes)); - nodes.emplace_back(Node{.step = std::move(step)}); + nodes.emplace_back(Node{.context = std::move(context), .step = std::move(step)}); root = &nodes.back(); for (auto & plan : plans) @@ -110,7 +113,7 @@ void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vectorgetName() + " to QueryPlan because " "step has no inputs, but QueryPlan is already initialized", ErrorCodes::LOGICAL_ERROR); - - nodes.emplace_back(Node{.step = std::move(step)}); + LOG_DEBUG(log, "Add step {} with context {}\n", step->getName(), static_cast(context.get())); + nodes.emplace_back(Node{.context = std::move(context), .step = std::move(step)}); root = &nodes.back(); return; } @@ -141,9 +144,9 @@ void QueryPlan::addStep(QueryPlanStepPtr step) "root header: " + root_header.dumpStructure() + "step header: " + step_header.dumpStructure(), ErrorCodes::LOGICAL_ERROR); - nodes.emplace_back(Node{.step = std::move(step), .children = {root}}); + nodes.emplace_back(Node{.context = std::move(context), .step = std::move(step), .children = {root}}); root->parent = &nodes.back(); - root = &nodes.back(); + root = root->parent; return; } @@ -268,7 +271,6 @@ void QueryPlan::buildStages(ContextPtr context) checkShuffle(frame.node, last_node, result); if (result.is_shuffle) { - ++stage_id; last_stage = createStage(stage_id, parent_stages, last_node, leaf_nodes); @@ -285,6 +287,17 @@ void QueryPlan::buildStages(ContextPtr context) { frame.node->num_parent_stages += last_node->num_parent_stages; frame.node->num_leaf_nodes_in_stage += last_node->num_leaf_nodes_in_stage; + /// If relationships between current node and all of its child nodes are shuffle, current node will have no context. + if (!frame.node->context && last_node->context) + { + frame.node->context = last_node->context; + LOG_DEBUG( + log, + "Set context({} <= {}) to {}", + frame.node->step->getName(), + last_node->step->getName(), + static_cast(frame.node->context.get())); + } } ++frame.visited_children; @@ -439,7 +452,7 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) "SystemContributors", "SystemLicenses", "SystemReplicatedMergeTreeSettings", "SystemMergeTreeSettings"}; - static std::unordered_set special_storages{"HDFS", "S3", "MySQL"}; + static std::unordered_set special_storages{"HDFS", "S3", "MySQL", "Memory"}; auto fillStage = [&store_replicas, &compute_replicas, this, &my_replica](Stage * stage) { @@ -555,7 +568,25 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) } } - /// Send plan fragment params. + /// Create query info. + GRPCQueryInfo query_info; + { + /// Fill with data shared among stages. + query_info.set_database(context->getCurrentDatabase()); + query_info.set_output_format("Native"); + + assert(!context->getClientInfo().distributed_query.empty()); + query_info.set_query(context->getClientInfo().distributed_query); + query_info.set_initial_query_id(initial_query_id); + + /// Fill changed settings. + for (const auto setting : context->getSettingsRef().allChanged()) + { + (*query_info.mutable_settings())[setting.getName()] = setting.getValueString(); + } + } + + /// Send query info. LOG_DEBUG(log, "===> Send stages."); for (auto & stage : stages) { @@ -611,25 +642,133 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) LOG_DEBUG(log, "Stage {} has {} workers.", stage.id, stage.workers.size()); assert(!stage.workers.empty()); - /// Create query info. - GRPCQueryInfo query_info; - query_info.set_database(context->getCurrentDatabase()); - query_info.set_output_format("Native"); - if (!context->getSelectQuery().empty()) - query_info.set_query(context->getSelectQuery()); /// For "insert into ... select" - else - query_info.set_query(context->getClientInfo().initial_query); + /// Fill with data related to each stage. query_info.set_query_id(context->generateQueryId()); - query_info.set_initial_query_id(initial_query_id); query_info.set_stage_id(stage.id); query_info.set_has_view_source(stage.has_view_source); query_info.set_has_input_function(stage.has_input_function); - for (const auto setting : context->getSettingsRef().allChanged()) + /// TODO: Not all stages need external tables, so choose the ones that are necessary, at least for leaf stages. + /// Fill external tables(reference from Connection.cpp: void Connection::sendExternalTablesData(ExternalTablesData & data)): + if (stage.is_leaf_stage) { - (*query_info.mutable_settings())[setting.getName()] = setting.getValueString(); + if (!stage.root_node->context) + LOG_DEBUG(log, "No need to prepare external tables data, because context is null."); + else + { + /// 1.Construct ExternalTablesData. + ExternalTablesData external_tables_data; + { + const auto & external_tables = stage.root_node->context->getExternalTables(); + LOG_DEBUG( + log, + "Prepare {} external tables using context {}.", + external_tables.size(), + static_cast(stage.root_node->context.get())); + for (const auto & table : external_tables) + { + StoragePtr cur = table.second; + + auto data = std::make_unique(); + data->table_name = table.first; + + LOG_DEBUG( + log, + "Prepare external table {} with {} ({}).", + data->table_name, + cur->getStorageID().getFullNameNotQuoted(), + cur->getName()); + { + SelectQueryInfo select_query_info; + auto metadata_snapshot = cur->getInMemoryMetadataPtr(); + QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage( + stage.root_node->context, QueryProcessingStage::Complete, metadata_snapshot, select_query_info); + + Pipe pipe = cur->read( + metadata_snapshot->getColumns().getNamesOfPhysical(), + metadata_snapshot, + select_query_info, + stage.root_node->context, + read_from_table_stage, + DEFAULT_BLOCK_SIZE, + 1); + + if (pipe.empty()) + { + data->pipe = std::make_unique( + std::make_shared(metadata_snapshot->getSampleBlock(), Chunk())); + } + else + { + data->pipe = std::make_unique(std::move(pipe)); + } + } + external_tables_data.emplace_back(std::move(data)); + } + } + + /// Fill external tables: + /// 2.Construct grpc data. + for (auto & data : external_tables_data) + { + Stopwatch watch; + clickhouse::grpc::ExternalTable external_table; + external_table.set_name(data->table_name); + external_table.set_format("Native"); + + assert(data->pipe); + + QueryPipelineBuilder pipeline_builder; + pipeline_builder.init(std::move(*data->pipe)); + data->pipe.reset(); + pipeline_builder.resize(1); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline_builder)); + + /// Fill columns name and type. + auto header = pipeline.getHeader(); + for (size_t i = 0; i < header.columns(); ++i) + { + ColumnWithTypeAndName column = header.safeGetByPosition(i); + clickhouse::grpc::NameAndType name_and_type; + name_and_type.set_name(column.name); + name_and_type.set_type(column.type->getName()); + external_table.mutable_columns()->Add(std::move(name_and_type)); + } + + /// Fill data. + std::optional write_buffer; + write_buffer.emplace(*external_table.mutable_data()); + std::shared_ptr output_format_processor = context->getOutputFormat("Native", *write_buffer, header); + output_format_processor->doWritePrefix(); + + Block block; + size_t rows = 0, bytes = 0; + auto executor = std::make_shared(pipeline); + while (executor->pull(block, 100)) + { + if (block) + { + rows += block.rows(); + bytes += block.bytes(); + output_format_processor->write(materializeBlock(block)); + } + } + output_format_processor->doWriteSuffix(); + LOG_DEBUG( + log, + "Fill external table {} with {} rows, {} bytes in {} sec.", + external_table.name(), + rows, + bytes, + watch.elapsedSeconds()); + + query_info.mutable_external_tables()->Add(std::move(external_table)); + } + } } + /// Fill parents id and sources. + query_info.clear_parent_sources(); for (const auto parent : stage.parents) { clickhouse::grpc::MapEntry entry; @@ -637,13 +776,18 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) entry.add_sources(*source); (*query_info.mutable_parent_sources())[parent->id] = entry; } + + /// Fill sinks. + query_info.clear_sinks(); for (const auto & sink : stage.sinks) { query_info.add_sinks(*sink); } + /// Send query info to each remote worker. for (const auto & worker : stage.workers) { + Stopwatch watch; query_info.set_node_id(*worker); LOG_DEBUG(log, "Remote plan fragment:\n{}", debugRemotePlanFragment(query_info.query(), *worker, initial_query_id, &stage)); @@ -657,7 +801,7 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) GRPCClient cli(*worker); auto result = cli.executePlanFragment(query_info); - LOG_DEBUG(log, "GRPCClient got result, exception code: {}, exception text: {}.", result.exception().code(), result.exception().display_text()); + LOG_DEBUG(log, "Finish sending GRPC query info in {} sec. Exception: (code {}) {}", watch.elapsedSeconds(), result.exception().code(), result.exception().display_text()); } } } @@ -884,13 +1028,20 @@ bool QueryPlan::buildDistributedPlan(ContextMutablePtr context) return false; } + if (context->getSkipDistributedPlan()) + { + LOG_DEBUG(log, "Skip building distributed plan, because skip_distributed_plan is true."); + return false; + } + { /// Print the original query plan. WriteBufferFromOwnString buf; buf << "------ Original Query Plan ------\n"; + buf << "SQL: " << context->getClientInfo().distributed_query << "\n"; QueryPlan::ExplainPlanOptions options{.header = true, .actions = true}; explainPlan(buf, options); - LOG_DEBUG(log, "Original query plan:\n{}", buf.str()); + LOG_DEBUG(log, "[{}] Original query plan:\n{}", static_cast(context.get()), buf.str()); } checkInitialized(); diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 9557237c12e0..e4064fe899c8 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -53,8 +53,8 @@ class QueryPlan QueryPlan(QueryPlan &&); QueryPlan & operator=(QueryPlan &&); - void unitePlans(QueryPlanStepPtr step, std::vector plans); - void addStep(QueryPlanStepPtr step); + void unitePlans(QueryPlanStepPtr step, std::vector plans, ContextPtr context = {}); + void addStep(QueryPlanStepPtr step, ContextPtr context = {}); bool isInitialized() const { return root != nullptr; } /// Tree is not empty bool isCompleted() const; /// Tree is not empty and root hasOutputStream() @@ -112,6 +112,7 @@ class QueryPlan /// Tree node. Step and it's children. struct Node { + ContextPtr context; QueryPlanStepPtr step; std::vector children = {}; Node * parent = nullptr; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index dd851f19906d..2ad5de48ecfc 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -120,7 +120,7 @@ void IStorage::read( else { auto read_step = std::make_unique(std::move(pipe), getName()); - query_plan.addStep(std::move(read_step)); + query_plan.addStep(std::move(read_step), context); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 22ec7ce6f53e..bd60da15c82b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -365,6 +365,8 @@ class MergeTreeData : public IStorage, public WithMutableContext bool attach, BrokenPartCallback broken_part_callback_ = [](const String &){}); + bool isRemote() const override { return true; } + /// Build a block of minmax and count values of a MergeTree table. These values are extracted /// from minmax_indices, the first expression of primary key, and part rows. /// diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 36ce3f25744b..8764f9465a60 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1237,6 +1237,7 @@ bool MutateTask::execute() bool MutateTask::prepare() { + LOG_DEBUG(&Poco::Logger::get("MutateTask"), "======== prepare"); MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry); if (ctx->future_part->parts.size() != 1) @@ -1253,32 +1254,35 @@ bool MutateTask::prepare() /// Allow mutations to work when force_index_by_date or force_primary_key is on. context_for_reading->setSetting("force_index_by_date", false); context_for_reading->setSetting("force_primary_key", false); - + LOG_DEBUG(&Poco::Logger::get("MutateTask"), "======== prepare: ----- 0"); for (const auto & command : *ctx->commands) { if (command.partition == nullptr || ctx->future_part->parts[0]->info.partition_id == ctx->data->getPartitionIDFromQuery( command.partition, context_for_reading)) ctx->commands_for_part.emplace_back(command); } - + LOG_DEBUG(&Poco::Logger::get("MutateTask"), "======== prepare: ----- 0.1"); if (ctx->source_part->isStoredOnDisk() && !isStorageTouchedByMutations( storage_from_source_part, ctx->metadata_snapshot, ctx->commands_for_part, Context::createCopy(context_for_reading))) { + LOG_DEBUG(&Poco::Logger::get("MutateTask"), "======== prepare: ----- 0.1.1"); LOG_TRACE(ctx->log, "Part {} doesn't change up to mutation version {}", ctx->source_part->name, ctx->future_part->part_info.mutation); promise.set_value(ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, "tmp_clone_", ctx->future_part->part_info, ctx->metadata_snapshot)); return false; } else { + LOG_DEBUG(&Poco::Logger::get("MutateTask"), "======== prepare: ----- 0.1.2"); LOG_TRACE(ctx->log, "Mutating part {} to mutation version {}", ctx->source_part->name, ctx->future_part->part_info.mutation); } - + LOG_DEBUG(&Poco::Logger::get("MutateTask"), "======== prepare: ----- 0.2"); MutationHelpers::splitMutationCommands(ctx->source_part, ctx->commands_for_part, ctx->for_interpreter, ctx->for_file_renames); ctx->stage_progress = std::make_unique(1.0); - + LOG_DEBUG(&Poco::Logger::get("MutateTask"), "======== prepare: ----- 1"); if (!ctx->for_interpreter.empty()) { + LOG_DEBUG(&Poco::Logger::get("MutateTask"), "======== prepare: !ctx->for_interpreter.empty()"); ctx->interpreter = std::make_unique( storage_from_source_part, ctx->metadata_snapshot, ctx->for_interpreter, context_for_reading, true); ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices(); @@ -1315,7 +1319,7 @@ bool MutateTask::prepare() const auto data_settings = ctx->data-> getSettings(); ctx->need_sync = needSyncPart(ctx->source_part->rows_count, ctx->source_part->getBytesOnDisk(), *data_settings); ctx->execute_ttl_type = ExecuteTTLType::NONE; - + LOG_DEBUG(&Poco::Logger::get("MutateTask"), "======== prepare: ----- 2"); if (ctx->mutating_pipeline.initialized()) ctx->execute_ttl_type = MergeTreeDataMergerMutator::shouldExecuteTTL(ctx->metadata_snapshot, ctx->interpreter->getColumnDependencies()); @@ -1325,11 +1329,12 @@ bool MutateTask::prepare() if (!isWidePart(ctx->source_part) || (ctx->mutation_kind == MutationsInterpreter::MutationKind::MUTATE_OTHER && ctx->interpreter && ctx->interpreter->isAffectingAllColumns())) { + LOG_DEBUG(&Poco::Logger::get("MutateTask"), "======== prepare: create MutateAllPartColumnsTask"); task = std::make_unique(ctx); } else /// TODO: check that we modify only non-key columns in this case. { - + LOG_DEBUG(&Poco::Logger::get("MutateTask"), "======== prepare: ----- 3"); /// We will modify only some of the columns. Other columns and key values can be copied as-is. for (const auto & name_type : ctx->updated_header.getNamesAndTypesList()) ctx->updated_columns.emplace(name_type.name); @@ -1356,7 +1361,7 @@ bool MutateTask::prepare() promise.set_value(ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, "tmp_clone_", ctx->future_part->part_info, ctx->metadata_snapshot)); return false; } - + LOG_DEBUG(&Poco::Logger::get("MutateTask"), "======== prepare: create MutateSomePartColumnsTask"); task = std::make_unique(ctx); } From a728dedec797f61ce22b19c9c4012eebfe656246 Mon Sep 17 00:00:00 2001 From: caspian Date: Wed, 12 Jan 2022 14:59:37 +0800 Subject: [PATCH 39/84] Add client support for recursive watch --- src/Common/ProfileEvents.cpp | 2 + src/Common/ZooKeeper/IKeeper.h | 28 ++++++ src/Common/ZooKeeper/TestKeeper.cpp | 68 +++++++++++++ src/Common/ZooKeeper/TestKeeper.h | 11 +++ src/Common/ZooKeeper/ZooKeeper.cpp | 89 +++++++++++++++++ src/Common/ZooKeeper/ZooKeeper.h | 15 +++ src/Common/ZooKeeper/ZooKeeperImpl.cpp | 129 +++++++++++++++++++++---- src/Common/ZooKeeper/ZooKeeperImpl.h | 20 ++++ 8 files changed, 342 insertions(+), 20 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index ee44b9eb9277..20e010dc8856 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -89,6 +89,8 @@ M(ZooKeeperCheck, "") \ M(ZooKeeperClose, "") \ M(ZooKeeperWatchResponse, "") \ + M(ZooKeeperAddWatch, "") \ + M(ZooKeeperRemomveWatches, "") \ M(ZooKeeperUserExceptions, "") \ M(ZooKeeperHardwareExceptions, "") \ M(ZooKeeperOtherExceptions, "") \ diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index f6ed8be1b99b..fa0e63f3c673 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -389,6 +389,8 @@ using SetCallback = std::function; using ListCallback = std::function; using CheckCallback = std::function; using MultiCallback = std::function; +using AddWatchCallback = std::function; +using RemoveWatchesCallback = std::function; /// For watches. @@ -412,6 +414,18 @@ enum Event NOTWATCHING = -2 }; +enum WatchMode +{ + Persistent = 0, + PersistentRecursive = 1, +}; + +enum WatchType +{ + Children = 1, + Data = 2, + Any = 3, +}; class Exception : public DB::Exception { @@ -510,6 +524,20 @@ class IKeeper const Requests & requests, MultiCallback callback) = 0; + /// Add a watch to the given znode using the given mode. Note: not all watch types can be set with this method. + /// Only the modes available in WatchMode can be set with this method. + virtual void addWatch( + const String &path, + WatchMode mode, + AddWatchCallback callback, + WatchCallback watch) = 0; + + /// For the given znode path, removes the specified watcher of given watchType. + virtual void removeWatches( + const String &path, + WatchType type, + RemoveWatchesCallback callback) = 0; + /// Expire session and finish all pending requests virtual void finalize(const String & reason) = 0; }; diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 065b1cf65ba9..0fee58b87fc6 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -125,6 +125,18 @@ struct TestKeeperListRequest final : ListRequest, TestKeeperRequest std::pair process(TestKeeper::Container & container, int64_t zxid) const override; }; +struct TestKeeperAddWatchRequest final : AddWatchRequest, TestKeeperRequest +{ + ResponsePtr createResponse() const override; + std::pair process(TestKeeper::Container & container, int64_t zxid) const override; +}; + +struct TestKeeperRemoveWatchesRequest final : RemoveWatchesRequest, TestKeeperRequest +{ + ResponsePtr createResponse() const override; + std::pair process(TestKeeper::Container & container, int64_t zxid) const override; +}; + struct TestKeeperCheckRequest final : CheckRequest, TestKeeperRequest { TestKeeperCheckRequest() = default; @@ -393,6 +405,28 @@ std::pair TestKeeperListRequest::process(TestKeeper::Containe return { std::make_shared(response), {} }; } +std::pair TestKeeperAddWatchRequest::process(TestKeeper::Container & container, int64_t) const +{ + AddWatchResponse response; + auto it = container.find(path); + if (it == container.end()) + { + response.error = Error::ZNONODE; + } + else + { + /// TODO add logic + } + return {std::make_shared(response), {}}; +} + +std::pair TestKeeperRemoveWatchesRequest::process(TestKeeper::Container &, int64_t) const +{ + // TODO add logic + RemoveWatchesResponse response; + return {std::make_shared(response), {}}; +} + std::pair TestKeeperCheckRequest::process(TestKeeper::Container & container, int64_t) const { CheckResponse response; @@ -472,6 +506,8 @@ ResponsePtr TestKeeperSetRequest::createResponse() const { return std::make_shar ResponsePtr TestKeeperListRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperCheckRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperMultiRequest::createResponse() const { return std::make_shared(); } +ResponsePtr TestKeeperAddWatchRequest::createResponse() const { return std::make_shared(); } +ResponsePtr TestKeeperRemoveWatchesRequest::createResponse() const { return std::make_shared(); } TestKeeper::TestKeeper(const String & root_path_, Poco::Timespan operation_timeout_) @@ -791,4 +827,36 @@ void TestKeeper::multi( pushRequest(std::move(request_info)); } +void TestKeeper::addWatch( + const String &path, + WatchMode mode, + AddWatchCallback callback, + WatchCallback watch) +{ + TestKeeperAddWatchRequest request; + request.path = path; + request.mode = mode; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; + request_info.watch = watch; + pushRequest(std::move(request_info)); +} + +/// For the given znode path, removes the specified watcher of given watchType. +void TestKeeper::removeWatches( + const String &path, + WatchType type, + RemoveWatchesCallback callback) +{ + TestKeeperRemoveWatchesRequest request; + request.path = path; + request.type = type; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; + pushRequest(std::move(request_info)); +} } diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index e57471341e8d..c6ad06796f73 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -83,6 +83,17 @@ class TestKeeper final : public IKeeper const Requests & requests, MultiCallback callback) override; + void addWatch( + const String &path, + WatchMode mode, + AddWatchCallback callback, + WatchCallback watch) override; + + void removeWatches( + const String &path, + WatchType type, + RemoveWatchesCallback callback) override; + void finalize(const String & reason) override; struct Node diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 4b817af049ab..594f926109e1 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -696,6 +696,67 @@ void ZooKeeper::tryRemoveRecursive(const std::string & path) tryRemove(path); } +void ZooKeeper::addWatch(const std::string & path, int mode, Coordination::WatchCallback watch_callback) +{ + check(tryAddWatch(path, Coordination::WatchMode(mode), watch_callback), path); +} + +Coordination::Error ZooKeeper::tryAddWatch(const std::string & path, Coordination::WatchMode mode, Coordination::WatchCallback watch_callback) +{ + Coordination::Error code = addWatchImpl(path, mode, watch_callback); + + if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) + throw KeeperException(code, path); + + return code; +} + +Coordination::Error ZooKeeper::addWatchImpl(const std::string & path, Coordination::WatchMode mode, Coordination::WatchCallback watch_callback) +{ + auto future_result = asyncTryAddRecursiveWatch(path, mode, watch_callback); + + if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) + { + impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Create), path)); + return Coordination::Error::ZOPERATIONTIMEOUT; + } + else + { + auto response = future_result.get(); + return response.error; + } +} + +void ZooKeeper::removeWatches(const std::string & path, int type) +{ + check(tryRemoveWatches(path, type), path); +} + +Coordination::Error ZooKeeper::tryRemoveWatches(const std::string & path, int type) +{ + Coordination::Error code = removeWatchesImpl(path, Coordination::WatchType(type)); + + if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) + throw KeeperException(code, path); + + return code; +} + +Coordination::Error ZooKeeper::removeWatchesImpl(const std::string & path, Coordination::WatchType type) +{ + auto future_result = asyncTryRemoveWatches(path, type); + + if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) + { + impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Create), path)); + return Coordination::Error::ZOPERATIONTIMEOUT; + } + else + { + auto response = future_result.get(); + return response.error; + } +} namespace { @@ -803,6 +864,34 @@ std::future ZooKeeper::asyncTryCreateNoThrow(const return future; } +std::future ZooKeeper::asyncTryAddRecursiveWatch(const std::string & path, Coordination::WatchMode mode, Coordination::WatchCallback watch_callback) +{ + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + auto callback = [promise](const Coordination::AddWatchResponse & response) mutable + { + promise->set_value(response); + }; + + impl->addWatch(path, mode, std::move(callback), watch_callback); + return future; +} + +std::future ZooKeeper::asyncTryRemoveWatches(const std::string & path, Coordination::WatchType type) +{ + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + auto callback = [promise](const Coordination::RemoveWatchesResponse & response) mutable + { + promise->set_value(response); + }; + + impl->removeWatches(path, type, std::move(callback)); + return future; +} + std::future ZooKeeper::asyncGet(const std::string & path, Coordination::WatchCallback watch_callback) { auto promise = std::make_shared>(); diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 8e015b1f331d..ec9c5f17f67c 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -204,6 +204,12 @@ class ZooKeeper /// Remove all children nodes (non recursive). void removeChildren(const std::string & path); + void addWatch(const std::string & path, int mode, Coordination::WatchCallback watch_callback); + Coordination::Error tryAddWatch(const std::string & path, Coordination::WatchMode mode, Coordination::WatchCallback watch_callback); + + void removeWatches(const std::string & path, int type); + Coordination::Error tryRemoveWatches(const std::string & path, int type); + using WaitCondition = std::function; /// Wait for the node to disappear or return immediately if it doesn't exist. @@ -274,6 +280,12 @@ class ZooKeeper /// * The node doesn't exist FutureGet asyncTryGet(const std::string & path); + using FutureAddWatch = std::future; + FutureAddWatch asyncTryAddRecursiveWatch(const std::string & path, Coordination::WatchMode mode, Coordination::WatchCallback watch_callback); + + using FutureRemoveWatches = std::future; + FutureRemoveWatches asyncTryRemoveWatches(const std::string & path, Coordination::WatchType type); + void finalize(const String & reason); void setZooKeeperLog(std::shared_ptr zk_log_); @@ -297,6 +309,9 @@ class ZooKeeper Coordination::Error multiImpl(const Coordination::Requests & requests, Coordination::Responses & responses); Coordination::Error existsImpl(const std::string & path, Coordination::Stat * stat_, Coordination::WatchCallback watch_callback); + Coordination::Error addWatchImpl(const std::string & path, Coordination::WatchMode mode, Coordination::WatchCallback watch_callback); + Coordination::Error removeWatchesImpl(const std::string & path, Coordination::WatchType mode); + std::unique_ptr impl; Strings hosts; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 09b2548f2f44..0a3038c1b8e3 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -35,6 +35,8 @@ namespace ProfileEvents extern const Event ZooKeeperBytesSent; extern const Event ZooKeeperBytesReceived; extern const Event ZooKeeperWatchResponse; + extern const Event ZooKeeperAddWatch; + extern const Event ZooKeeperRemomveWatches; } namespace CurrentMetrics @@ -704,29 +706,62 @@ void ZooKeeper::receiveEvent() { const WatchResponse & watch_response = dynamic_cast(response_); - std::lock_guard lock(watches_mutex); + { + std::lock_guard lock(watches_mutex); + + auto it = watches.find(watch_response.path); + if (it == watches.end()) + { + /// This is Ok. + /// Because watches are identified by path. + /// And there may exist many watches for single path. + /// And watch is added to the list of watches on client side + /// slightly before than it is registered by the server. + /// And that's why new watch may be already fired by old event, + /// but then the server will actually register new watch + /// and will send event again later. + } + else + { + for (auto & callback : it->second) + if (callback) + callback(watch_response); /// NOTE We may process callbacks not under mutex. - auto it = watches.find(watch_response.path); - if (it == watches.end()) + CurrentMetrics::sub(CurrentMetrics::ZooKeeperWatch, it->second.size()); + watches.erase(it); + } + } + /// Maybe it's a persistent watch { - /// This is Ok. - /// Because watches are identified by path. - /// And there may exist many watches for single path. - /// And watch is added to the list of watches on client side - /// slightly before than it is registered by the server. - /// And that's why new watch may be already fired by old event, - /// but then the server will actually register new watch - /// and will send event again later. + std::lock_guard lock(persistent_watches_mutex); + auto it = persistent_watches.find(watch_response.path); + if (it != persistent_watches.end()) + { + for (auto & callback : it->second) + if (callback) + callback(watch_response); + } } - else { - for (auto & callback : it->second) - if (callback) - callback(watch_response); /// NOTE We may process callbacks not under mutex. + std::lock_guard lock(persistent_recursive_watches_mutex); + for (auto path = watch_response.path; !path.empty(); ) + { + auto it = persistent_recursive_watches.find(path); + if (it != persistent_recursive_watches.end()) + { + for (auto & callback : it->second) + if (callback) + callback(watch_response); + } - CurrentMetrics::sub(CurrentMetrics::ZooKeeperWatch, it->second.size()); - watches.erase(it); + if (path.rfind("/") != std::string::npos) + { + path = path.substr(0, path.rfind('/')); + trimRight(path, '/'); + } + } } + }; } else @@ -788,9 +823,28 @@ void ZooKeeper::receiveEvent() /// The key of wathces should exclude the root_path String req_path = request_info.request->getPath(); - removeRootPath(req_path, root_path); - std::lock_guard lock(watches_mutex); - watches[req_path].emplace_back(std::move(request_info.watch)); + if (request_info.request->getOpNum() == OpNum::AddWatch) + { + ZooKeeperAddWatchRequest & request = dynamic_cast(*request_info.request); + if (request.mode == int32_t(WatchMode::Persistent)) + { + std::lock_guard lock(persistent_watches_mutex); + persistent_watches[req_path].emplace_back(std::move(request_info.watch)); + } + else if (request.mode == int32_t(WatchMode::PersistentRecursive)) + { /// + std::lock_guard lock(persistent_recursive_watches_mutex); + persistent_recursive_watches[req_path].emplace_back(std::move(request_info.watch)); + } + else + throw Exception("Invalid watch mode in addWatch request: " + DB::toString(request.mode), Error::ZBADARGUMENTS); + } + else + { + removeRootPath(req_path, root_path); + std::lock_guard lock(watches_mutex); + watches[req_path].emplace_back(std::move(request_info.watch)); + } } } @@ -1200,6 +1254,41 @@ void ZooKeeper::multi( ProfileEvents::increment(ProfileEvents::ZooKeeperMulti); } +void ZooKeeper::addWatch( + const String &path, + WatchMode mode, + AddWatchCallback callback, + WatchCallback watch) +{ + ZooKeeperAddWatchRequest request; + request.path = path; + request.mode = int32_t(mode); + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; + request_info.watch = watch; + + pushRequest(std::move(request_info)); + ProfileEvents::increment(ProfileEvents::ZooKeeperAddWatch); +} + +void ZooKeeper::removeWatches( + const String &path, + WatchType type, + RemoveWatchesCallback callback) +{ + ZooKeeperRemoveWatchesRequest request; + request.path = path; + request.type = int32_t(type); + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; + + pushRequest(std::move(request_info)); + ProfileEvents::increment(ProfileEvents::ZooKeeperRemomveWatches); +} void ZooKeeper::close() { diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index b87469bd3399..d1019b7b049b 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -175,6 +175,20 @@ class ZooKeeper final : public IKeeper const Requests & requests, MultiCallback callback) override; + /// Add a watch to the given znode using the given mode. Note: not all watch types can be set with this method. + /// Only the modes available in WatchMode can be set with this method. + void addWatch( + const String &path, + WatchMode mode, + AddWatchCallback callback, + WatchCallback watch) override; + + /// For the given znode path, removes the specified watcher of given watchType. + void removeWatches( + const String &path, + WatchType type, + RemoveWatchesCallback callback) override; + /// Without forcefully invalidating (finalizing) ZooKeeper session before /// establishing a new one, there was a possibility that server is using /// two ZooKeeper sessions simultaneously in different parts of code. @@ -237,6 +251,12 @@ class ZooKeeper final : public IKeeper Watches watches; std::mutex watches_mutex; + Watches persistent_watches; + std::mutex persistent_watches_mutex; + + Watches persistent_recursive_watches; + std::mutex persistent_recursive_watches_mutex; + ThreadFromGlobalPool send_thread; ThreadFromGlobalPool receive_thread; From 026702e8e2f2683f75ed0fa6196fb90df3f9161e Mon Sep 17 00:00:00 2001 From: madianjun Date: Tue, 11 Jan 2022 19:50:47 +0800 Subject: [PATCH 40/84] Eliminate the last stage when it has only one source worker which is also a compute worker. --- src/Processors/QueryPlan/QueryPlan.cpp | 105 +++++++++++++++---------- src/Processors/QueryPlan/QueryPlan.h | 3 +- 2 files changed, 67 insertions(+), 41 deletions(-) diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 7a88ad54c517..28961b395e29 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -402,7 +402,7 @@ void QueryPlan::debugStages() LOG_DEBUG(log, "===> Print Stages:\n{}", buf.str()); } -void QueryPlan::scheduleStages(ContextMutablePtr context) +bool QueryPlan::scheduleStages(ContextMutablePtr context) { LOG_DEBUG(log, "===> Schedule stages."); /// Use initial query id to build the plan fragment id. @@ -454,7 +454,8 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) static std::unordered_set special_storages{"HDFS", "S3", "MySQL", "Memory"}; - auto fillStage = [&store_replicas, &compute_replicas, this, &my_replica](Stage * stage) + bool is_result_stage_moved_forward = false; + auto fillStage = [&store_replicas, &compute_replicas, this, &my_replica, &is_result_stage_moved_forward](Stage * stage) { /// Leaf stage. if (stage->is_leaf_stage) @@ -517,11 +518,15 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) { auto * parent = stage->parents.front(); if (parent->workers.size() == 1 && !parent->parents.empty()) - parent->workers.front() = stage->workers.front(); - - /// Another solution: merge result stage into its parent stage. - //LOG_DEBUG(log, "Result stage {} moves forward to parent stage {}.", result_stage->id, result_stage->parents[0]->id); - //result_stage = result_stage->parents[0]; + { + /// Use result stage's parent as result stage. + LOG_DEBUG(log, "Move result stage {} forward to parent stage {}.", result_stage->id, parent->id); + assert(result_stage == &stages.back()); + result_stage = parent; + root = parent->root_node; + stages.pop_back(); + is_result_stage_moved_forward = true; + } } return; } @@ -594,40 +599,56 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) if (&stage == result_stage) { assert(!result_stage->parents.empty()); - /// Clear query plan tree. - root = nullptr; - - for (const auto parent_stage : result_stage->parents) + if (is_result_stage_moved_forward) { - const Node * parent_stage_node = parent_stage->root_node; - const auto & header = parent_stage_node->step->getOutputStream().header; - assert(header); - LOG_DEBUG( - log, - "Take the output stream header of {}: {}, header columns: {}.", - parent_stage_node->step->getName(), - parent_stage_node->step->getStepDescription(), - header.columns()); - - auto distributed_source_step = std::make_unique( - header, - parent_stage->workers, - initial_query_id, - result_stage->id, - parent_stage->id, - *result_stage->workers.front(), - false, - context); - /// TODO: Improve to support adding multiple distributed_source_step, such as Union operator. - addStep(std::move(distributed_source_step)); + Context::QueryPlanFragmentInfo query_plan_fragment_info{ + .initial_query_id = initial_query_id, + .stage_id = stage.id, + .node_id = my_replica + }; + for (const auto parent : stage.parents) + { + query_plan_fragment_info.parent_sources[parent->id] = parent->workers; + } + query_plan_fragment_info.sinks = stage.sinks; + context->setQueryPlanFragmentInfo(query_plan_fragment_info); } + else { - /// Only for debug. - LOG_DEBUG( - log, - "Result plan fragment:\n{}", - debugLocalPlanFragment( - initial_query_id, result_stage->id, *result_stage->workers.front(), std::vector{root})); + /// Clear query plan tree. + root = nullptr; + + for (const auto parent_stage : result_stage->parents) + { + const Node * parent_stage_node = parent_stage->root_node; + const auto & header = parent_stage_node->step->getOutputStream().header; + assert(header); + LOG_DEBUG( + log, + "Take the output stream header of {}: {}, header columns: {}.", + parent_stage_node->step->getName(), + parent_stage_node->step->getStepDescription(), + header.columns()); + + auto distributed_source_step = std::make_unique( + header, + parent_stage->workers, + initial_query_id, + result_stage->id, + parent_stage->id, + *result_stage->workers.front(), + false, + context); + addStep(std::move(distributed_source_step)); + } + { + /// Only for debug. + LOG_DEBUG( + log, + "Result plan fragment:\n{}", + debugLocalPlanFragment( + initial_query_id, result_stage->id, *result_stage->workers.front(), std::vector{root})); + } } continue; } @@ -804,6 +825,8 @@ void QueryPlan::scheduleStages(ContextMutablePtr context) LOG_DEBUG(log, "Finish sending GRPC query info in {} sec. Exception: (code {}) {}", watch.elapsedSeconds(), result.exception().code(), result.exception().display_text()); } } + + return is_result_stage_moved_forward; } void QueryPlan::buildPlanFragment(ContextPtr context) @@ -812,7 +835,8 @@ void QueryPlan::buildPlanFragment(ContextPtr context) int my_stage_id = query_distributed_plan_info.stage_id; LOG_DEBUG( log, - "===> Build plan fragment: stage {}, has {} parent stages.", + "===> Build plan fragment: {} stage {}, has {} parent stages.", + (result_stage ? (my_stage_id == result_stage->id ? "result": "no-result") : "no-result"), my_stage_id, query_distributed_plan_info.parent_sources.size()); @@ -1049,7 +1073,8 @@ bool QueryPlan::buildDistributedPlan(ContextMutablePtr context) if (context->isInitialQuery()) { buildStages(context); - scheduleStages(context); + if (scheduleStages(context)) + buildPlanFragment(context); } else { diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index e4064fe899c8..433d8c2ee0a4 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -66,7 +66,8 @@ class QueryPlan void buildStages(ContextPtr context); /// Used by initial node. void debugStages(); - void scheduleStages(ContextMutablePtr context); /// Used by initial node. + /// Return true if result stage is moved forward. + bool scheduleStages(ContextMutablePtr context); /// Used by initial node. void buildPlanFragment(ContextPtr context); /// Used by non-initial nodes. bool buildDistributedPlan(ContextMutablePtr context); From d3d83bfc115962444e70ebe999353574b6826c66 Mon Sep 17 00:00:00 2001 From: madianjun Date: Wed, 12 Jan 2022 15:11:50 +0800 Subject: [PATCH 41/84] Fix bug about materialized view --- src/Interpreters/Context.cpp | 22 ++++++++--------- src/Interpreters/Context.h | 4 ++-- src/Processors/QueryPlan/QueryPlan.cpp | 33 +++++++++++++++++++++----- src/Server/GRPCServer.cpp | 15 ++++++++---- 4 files changed, 51 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index f3d3f088757d..9fe4826af3ad 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -149,7 +149,7 @@ struct ContextSharedPart mutable std::mutex external_dictionaries_mutex; mutable std::mutex external_user_defined_executable_functions_mutex; mutable std::mutex external_models_mutex; - mutable std::mutex initial_query_contexts_mutex; + mutable std::mutex initial_contexts_mutex; /// Separate mutex for storage policies. During server startup we may /// initialize some important storages (system logs with MergeTree engine) /// under context lock. @@ -189,7 +189,7 @@ struct ContextSharedPart mutable std::optional external_dictionaries_loader; mutable std::optional external_user_defined_executable_functions_loader; mutable std::optional external_models_loader; - std::unordered_map initial_query_contexts; + std::unordered_map initial_contexts; /// Retrieve view source or input function in different plan fragments. ExternalLoaderXMLConfigRepository * external_models_config_repository = nullptr; scope_guard models_repository_guard; @@ -1059,20 +1059,20 @@ StoragePtr Context::getViewSource() const return view_source; } -void Context::addInitialQueryContext(const String & plan_fragment_id, const ContextPtr context) +void Context::addInitialContext(const String & plan_fragment_id, const ContextPtr & context) { - std::lock_guard lock(shared->initial_query_contexts_mutex); - shared->initial_query_contexts[plan_fragment_id] = context; + std::lock_guard lock(shared->initial_contexts_mutex); + shared->initial_contexts[plan_fragment_id] = context; } -ContextPtr Context::getInitialQueryContext(const String & plan_fragment_id) const +ContextPtr Context::getInitialContext(const String & plan_fragment_id) const { - std::lock_guard lock(shared->initial_query_contexts_mutex); - auto it = shared->initial_query_contexts.find(plan_fragment_id); - if (it == shared->initial_query_contexts.end()) - throw Exception(ErrorCodes::BAD_GET, "There is no initial query context for {}", plan_fragment_id); + std::lock_guard lock(shared->initial_contexts_mutex); + auto it = shared->initial_contexts.find(plan_fragment_id); + if (it == shared->initial_contexts.end()) + throw Exception(ErrorCodes::BAD_GET, "There is no initial context for {}", plan_fragment_id); ContextPtr initial_query_context = it->second; - shared->initial_query_contexts.erase(it); + shared->initial_contexts.erase(it); return initial_query_context; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index ab58d598da0b..a03c002d878d 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -531,8 +531,8 @@ class Context: public std::enable_shared_from_this void addViewSource(const StoragePtr & storage); StoragePtr getViewSource() const; - void addInitialQueryContext(const String & plan_fragment_id, const ContextPtr context); - ContextPtr getInitialQueryContext(const String & plan_fragment_id) const; + void addInitialContext(const String & plan_fragment_id, const ContextPtr & context); + ContextPtr getInitialContext(const String & plan_fragment_id) const; bool isStandaloneMode() const; String getCurrentDatabase() const; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 28961b395e29..e0045f223005 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -511,13 +511,13 @@ bool QueryPlan::scheduleStages(ContextMutablePtr context) if (stage == result_stage) { stage->workers.emplace_back(std::make_shared(my_replica)); - /// Optimize: - /// the result stage has only one parent worker and its parent stage is not leaf stage, - /// place its parent stage on the same worker as result stage. + + /// Maybe the last stage can be eliminated. if (stage->parents.size() == 1) { auto * parent = stage->parents.front(); - if (parent->workers.size() == 1 && !parent->parents.empty()) + /// Parent stage will be scheduled on the same worker as result stage. + if (parent->workers.size() == 1 && *(parent->workers.front()) == my_replica) { /// Use result stage's parent as result stage. LOG_DEBUG(log, "Move result stage {} forward to parent stage {}.", result_stage->id, parent->id); @@ -816,8 +816,29 @@ bool QueryPlan::scheduleStages(ContextMutablePtr context) { const String & plan_fragment_id = query_info.initial_query_id() + "/" + toString(query_info.stage_id()) + "/" + query_info.node_id(); - context->addInitialQueryContext(plan_fragment_id, context->getQueryContext()); - LOG_DEBUG(log, "Store initial query context for plan fragment {}, because has {}.", plan_fragment_id, (stage.has_view_source ? "view source" : "input function")); + if (stage.has_view_source) + { + const auto & view_source = context->getViewSource(); + assert(view_source); + LOG_DEBUG( + log, + "Store initial context {} for plan fragment {}, because has view source: {}({}).", + static_cast(context.get()), + plan_fragment_id, + view_source->getStorageID().getFullNameNotQuoted(), + view_source->getName()); + context->addInitialContext(plan_fragment_id, context); + } + else + { + LOG_DEBUG( + log, + "Store initial context {} for plan fragment {}, because has input function.", + static_cast(context->getQueryContext().get()), + plan_fragment_id); + context->addInitialContext(plan_fragment_id, context->getQueryContext()); + } + } GRPCClient cli(*worker); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 87e3bf0dbe91..15746c40ecf8 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1262,15 +1262,22 @@ namespace if (query_info->has_view_source() || query_info->has_input_function()) { - const auto & initial_query_context = query_context->getInitialQueryContext(plan_fragment_id); + const auto & initial_context = query_context->getInitialContext(plan_fragment_id); if (query_info->has_view_source()) { - LOG_DEBUG(log, "Restore view source for plan fragment {}", plan_fragment_id); - query_context->addViewSource(initial_query_context->getViewSource()); + const auto & view_source = initial_context->getViewSource(); + assert(view_source); + LOG_DEBUG( + log, + "Restore view source {}({}) for plan fragment {}", + view_source->getStorageID().getFullNameNotQuoted(), + view_source->getName(), + plan_fragment_id); + query_context->addViewSource(view_source); } else if (query_info->has_input_function()) { - query_context->setQueryContext(std::const_pointer_cast(initial_query_context)); + query_context->setQueryContext(std::const_pointer_cast(initial_context)); } } } From 613d931141e02015e3978432b222d35f92e7e34a Mon Sep 17 00:00:00 2001 From: madianjun Date: Thu, 13 Jan 2022 10:26:29 +0800 Subject: [PATCH 42/84] Fix the checking of join on keys after join sql is rewritten --- src/Interpreters/CollectJoinOnKeysVisitor.cpp | 3 +++ src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- .../InterpreterSelectWithUnionQuery.cpp | 13 +++++++------ 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/src/Interpreters/CollectJoinOnKeysVisitor.cpp index 9715af01a0ab..d88c4e2db7dc 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -243,6 +243,9 @@ JoinIdentifierPos CollectJoinOnKeysMatcher::getTableForIdentifiers(const ASTPtr in_right_table = false; in_left_table = !in_right_table; } + /// Consider it's in left table if has no table name. + else if (identifier->isShort()) + in_right_table = false; else throw Exception("Column '" + name + "' is ambiguous", ErrorCodes::AMBIGUOUS_COLUMN_NAME); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index b8e35a6eb96a..8c2668513b22 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -439,7 +439,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( std::move(subquery_for_sets), std::move(prepared_sets)); - /// Process SQL just like "SELECT ... FROM _temporary_and_external_tables.`_tmp_fbe82e3a-1815-4563-bbe8-2e3a1815e563`" + /// For SQL just like "SELECT ... FROM _temporary_and_external_tables.`_tmp_fbe82e3a-1815-4563-bbe8-2e3a1815e563`" if (!query_analyzer->getExternalTables().empty()) context->setSkipDistributedPlan(true); @@ -509,7 +509,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( result_header = getSampleBlockImpl(); /// Maybe subquery has been rewritten with "_subqueryX", so reset distributed_query. String maybe_rewritten_query = queryToString(query_ptr); - LOG_DEBUG(log, "[{}] Rewrite \"{}\" to \"{}\"", static_cast(context.get()), context->getClientInfo().distributed_query, maybe_rewritten_query); + LOG_DEBUG(log, "[{}] Rewrite from \"{}\" to \"{}\"", static_cast(context.get()), context->getClientInfo().distributed_query, maybe_rewritten_query); context->getClientInfo().distributed_query = std::move(maybe_rewritten_query); }; diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 897d4bad2995..75dbfde2ae2f 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -167,24 +167,24 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( options.ignore_limits |= all_nested_ignore_limits; options.ignore_quota |= all_nested_ignore_quota; - context->getClientInfo().distributed_query.clear(); + String rewritten_query; for (size_t query_num = 0; query_num < num_children; ++query_num) { - context->getClientInfo().distributed_query += nested_interpreters[query_num]->getContext()->getClientInfo().distributed_query; + rewritten_query += nested_interpreters[query_num]->getContext()->getClientInfo().distributed_query; if (query_num < num_children - 1) { if (ast->union_mode == ASTSelectWithUnionQuery::Mode::Unspecified) { - context->getClientInfo().distributed_query += " UNION "; + rewritten_query += " UNION "; } else if (ast->union_mode == ASTSelectWithUnionQuery::Mode::ALL) { - context->getClientInfo().distributed_query += " UNION ALL "; + rewritten_query += " UNION ALL "; } else if (ast->union_mode == ASTSelectWithUnionQuery::Mode::DISTINCT) { - context->getClientInfo().distributed_query += " UNION DISTINCT "; + rewritten_query += " UNION DISTINCT "; } else { @@ -192,7 +192,8 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( } } } - LOG_DEBUG(log, "[{}] Rewrite to: {}", static_cast(context.get()), context->getClientInfo().distributed_query); + LOG_DEBUG(log, "[{}] Rewrite from \"{}\" to: \"{}\"", static_cast(context.get()), context->getClientInfo().distributed_query, rewritten_query); + context->getClientInfo().distributed_query = std::move(rewritten_query); } Block InterpreterSelectWithUnionQuery::getCommonHeaderForUnion(const Blocks & headers) From b8654a70512322f0b32184377a81f63f53bfa75d Mon Sep 17 00:00:00 2001 From: madianjun Date: Thu, 13 Jan 2022 16:28:42 +0800 Subject: [PATCH 43/84] Set prefer_global_in_and_join default to true --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9c8f8f85fb5d..346651a1b7e1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -270,7 +270,7 @@ class IColumn; M(String, metrics_perf_events_list, "", "Comma separated list of perf metrics that will be measured throughout queries' execution. Empty means all events. See PerfEventInfo in sources for the available events.", 0) \ M(Float, opentelemetry_start_trace_probability, 0., "Probability to start an OpenTelemetry trace for an incoming query.", 0) \ M(Bool, prefer_column_name_to_alias, false, "Prefer using column names instead of aliases if possible.", 0) \ - M(Bool, prefer_global_in_and_join, false, "If enabled, all IN/JOIN operators will be rewritten as GLOBAL IN/JOIN. It's useful when the to-be-joined tables are only available on the initiator and we need to always scatter their data on-the-fly during distributed processing with the GLOBAL keyword. It's also useful to reduce the need to access the external sources joining external tables.", 0) \ + M(Bool, prefer_global_in_and_join, true, "If enabled, all IN/JOIN operators will be rewritten as GLOBAL IN/JOIN. It's useful when the to-be-joined tables are only available on the initiator and we need to always scatter their data on-the-fly during distributed processing with the GLOBAL keyword. It's also useful to reduce the need to access the external sources joining external tables.", 0) \ \ \ /** Limits during query execution are part of the settings. \ From 3d0a02fa4dd88cbf3124e3b703192b8c6374578d Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Tue, 11 Jan 2022 19:46:49 +0800 Subject: [PATCH 44/84] Using grpc server exception code as client code --- src/Client/GRPCClient.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/GRPCClient.cpp b/src/Client/GRPCClient.cpp index 914e2801e25a..d41b75f63cf7 100644 --- a/src/Client/GRPCClient.cpp +++ b/src/Client/GRPCClient.cpp @@ -82,7 +82,7 @@ Block GRPCClient::read() addr, result.exception().code(), result.exception().display_text()); - throw Exception(result.exception().display_text(), ErrorCodes::GRPC_READ_ERROR, true); + throw Exception(result.exception().display_text(), result.exception().code(), true); } if (result.output().size() == 0) From 6ff37dc9f749c8520d5ae2fc602987125b957f6f Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Fri, 7 Jan 2022 16:40:17 +0800 Subject: [PATCH 45/84] Ignore znode of database lock when loading metadata from meta-service --- src/Core/Defines.h | 1 + src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Interpreters/loadMetadata.cpp | 2 ++ 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 459dc0132a34..b7189b1906e1 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -2,6 +2,7 @@ #include +#define DATABASE_LOCK_SUFFIX "_0X5F3759DF_DB_LOCK" #define DEFAULT_ZOOKEEPER_METADATA_PATH "/clickhouse/metadata" #define CLUSTER_TYPE_COMPUTE "compute" #define CLUSTER_TYPE_STORE "store" diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 0baee4c26bb4..470f850f2167 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -991,7 +991,7 @@ DistributedDDLGuard::DistributedDDLGuard(zkutil::ZooKeeperPtr zookeeper_, const { auto path = fs::path(DEFAULT_ZOOKEEPER_METADATA_PATH); if (table_name_.empty()) - path = path / (database_name_ + "_lock"); + path = path / (database_name_ + DATABASE_LOCK_SUFFIX); else path = path / database_name_ / "metadata" / (table_name_ + "_lock"); hold_path = zookeeper->create(path, "", zkutil::CreateMode::Ephemeral); diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 9166bb3b5575..348f227cf72b 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -174,6 +174,8 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam Strings database_names = zookeeper->getChildren(zookeeper_path); for (const auto & database_name : database_names) { + if (database_name.ends_with(DATABASE_LOCK_SUFFIX)) + continue; String query = zookeeper->get(zookeeper_path / database_name); database_name_to_metadata.emplace(unescapeForFileName(database_name), query); } From 5797de761ab1c2b3037b4204cfdeb0a235f55384 Mon Sep 17 00:00:00 2001 From: caspian Date: Fri, 14 Jan 2022 05:50:08 +0000 Subject: [PATCH 46/84] Fix watch bug --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 13 ++++++++++--- src/Coordination/KeeperStorage.cpp | 2 +- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 0a3038c1b8e3..3472995bee4c 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -754,10 +754,17 @@ void ZooKeeper::receiveEvent() callback(watch_response); } - if (path.rfind("/") != std::string::npos) + auto pos = path.rfind("/"); + if (pos != std::string::npos) { - path = path.substr(0, path.rfind('/')); - trimRight(path, '/'); + if (pos == 0 && path.length() > 1) { + path = "/"; + } + else + { + path = path.substr(0, pos); + trimRight(path, '/'); + } } } } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index d3b2537e26f5..4e44b54789ae 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -163,7 +163,7 @@ static void triggerWatches(const String rawPath, const String path, int depth, C // in persistent-recursive mode, children changed event is redundant, so the node add/delete event path should be the raw path watch_response->path = rawPath; } - else if (depth == 0 || (depth == 1 && watcherType == WatcherType::Data)) + else if (depth == 0 || (depth == 1 && watcherType == WatcherType::Children)) matched = true; } From ace5f566a1703b64a16c3238f29da3e7457ec4cd Mon Sep 17 00:00:00 2001 From: madianjun Date: Thu, 13 Jan 2022 17:16:35 +0800 Subject: [PATCH 47/84] Fix SELECT WITH TOTALS/ROLLUP/CUBE --- src/Client/GRPCClient.cpp | 35 ++++++++--- src/Client/GRPCClient.h | 10 +++- src/Interpreters/InterpreterSelectQuery.cpp | 3 +- .../InterpreterSelectWithUnionQuery.cpp | 11 ++++ .../QueryPlan/DistributedSourceStep.cpp | 37 +++++++++++- .../QueryPlan/DistributedSourceStep.h | 3 + src/Processors/QueryPlan/QueryPlan.cpp | 58 +++++++++++-------- src/Processors/QueryPlan/QueryPlan.h | 23 +++++++- src/Processors/Sources/DistributedSource.cpp | 50 +++++++++++++++- src/Processors/Sources/DistributedSource.h | 35 ++++++++++- .../DistributedSourceExecutor.cpp | 28 +++++++-- src/QueryPipeline/DistributedSourceExecutor.h | 6 ++ src/Storages/IStorage.cpp | 4 +- 13 files changed, 253 insertions(+), 50 deletions(-) diff --git a/src/Client/GRPCClient.cpp b/src/Client/GRPCClient.cpp index d41b75f63cf7..3bfe140c6779 100644 --- a/src/Client/GRPCClient.cpp +++ b/src/Client/GRPCClient.cpp @@ -66,7 +66,7 @@ void GRPCClient::prepareRead(const GRPCTicket & ticket_) inner_context = std::make_unique(ch, ctx, stub, reader); } -Block GRPCClient::read() +GRPCClient::MessageType GRPCClient::read(Block & block) { assert(inner_context); @@ -85,14 +85,33 @@ Block GRPCClient::read() throw Exception(result.exception().display_text(), result.exception().code(), true); } - if (result.output().size() == 0) - return {}; /// Read EOF + /// Note: totals and extremes are not used and not tested yet. + if (result.totals().size() > 0) + { + ReadBufferFromString b(result.totals()); + NativeReader reader(b, 0); + block = reader.read(); + LOG_DEBUG(log, "Read totals from {} success, result size: {}, block rows: {}.", addr, result.output().size(), block.rows()); + return MessageType::Totals; + } - ReadBufferFromString b(result.output()); - NativeReader reader(b, 0 /* server_revision_ */); - Block block = reader.read(); - LOG_DEBUG(log, "Read from {} success, result size: {}, block rows: {}.", addr, result.output().size(), block.rows()); - return block; + if (result.extremes().size() > 0) + { + ReadBufferFromString b(result.extremes()); + NativeReader reader(b, 0); + block = reader.read(); + LOG_DEBUG(log, "Read extremes from {} success, result size: {}, block rows: {}.", addr, result.output().size(), block.rows()); + return MessageType::Extremes; + } + + if (!result.output().empty()) + { + ReadBufferFromString b(result.output()); + NativeReader reader(b, 0); + block = reader.read(); + LOG_DEBUG(log, "Read data from {} success, result size: {}, block rows: {}.", addr, result.output().size(), block.rows()); + } + return MessageType::Data; } throw Exception( diff --git a/src/Client/GRPCClient.h b/src/Client/GRPCClient.h index 0d4bcd4aff02..17ef401cbd54 100644 --- a/src/Client/GRPCClient.h +++ b/src/Client/GRPCClient.h @@ -25,6 +25,14 @@ using ReadDataCallback = std::function; class GRPCClient { +public: + enum MessageType + { + Data = 1, + Totals = 2, + Extremes = 3, + MAX = Extremes, + }; public: GRPCClient(const String & addr_); ~GRPCClient() = default; @@ -37,7 +45,7 @@ class GRPCClient /// Try to read a block from remote. /// If got EOF, an empty block will be returned, you can use if (!block) to check it. - Block read(); + MessageType read(Block & bock); /// Cancel plan fragment (ticket associated with the prepareRead) void cancel(); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8c2668513b22..0a1418301533 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1527,7 +1527,8 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan( auto read_from_pipe = std::make_unique(std::move(pipe)); read_from_pipe->setStepDescription("Read from NullSource"); - query_plan.addStep(std::move(read_from_pipe), context_); + InterpreterParamsPtr interpreter_params = std::make_shared(context_, query_info.query->as()); + query_plan.addStep(std::move(read_from_pipe), std::move(interpreter_params)); if (query_info.projection) { diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 75dbfde2ae2f..ed5c27d2a6b3 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -356,6 +356,17 @@ BlockIO InterpreterSelectWithUnionQuery::execute() QueryPlan query_plan; buildQueryPlan(query_plan); + { + /// Print the original query plan for debugging distributed table. + /// TODO: This will be removed in the future. + WriteBufferFromOwnString buf; + buf << "------ DEBUG Query Plan ------\n"; + buf << "SQL: " << context->getClientInfo().distributed_query << "\n"; + QueryPlan::ExplainPlanOptions options{.header = true, .actions = true}; + query_plan.explainPlan(buf, options); + LOG_DEBUG(log, "[{}] DEBUG query plan:\n{}", static_cast(context.get()), buf.str()); + } + bool is_built = query_plan.buildDistributedPlan(context); QueryPlanOptimizationSettings optimization_settings{.optimize_plan = !is_built}; diff --git a/src/Processors/QueryPlan/DistributedSourceStep.cpp b/src/Processors/QueryPlan/DistributedSourceStep.cpp index 564cb534019e..45cb0463ee74 100644 --- a/src/Processors/QueryPlan/DistributedSourceStep.cpp +++ b/src/Processors/QueryPlan/DistributedSourceStep.cpp @@ -1,10 +1,40 @@ #include #include #include +#include +#include +#include +#include +#include namespace DB { +/// Copy from ReadFromRemote.cpp +static ActionsDAGPtr getConvertingDAG(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(), + ActionsDAG::MatchColumnsMode::Name, + true); +} + +/// Copy from ReadFromRemote.cpp +static void addConvertingActions(Pipe & pipe, const Block & header) +{ + if (blocksHaveEqualStructure(pipe.getHeader(), header)) + return; + + auto convert_actions = std::make_shared(getConvertingDAG(pipe.getHeader(), header)); + pipe.addSimpleTransform([&](const Block & cur_header, Pipe::StreamType) -> ProcessorPtr { + return std::make_shared(cur_header, convert_actions); + }); +} + DistributedSourceStep::DistributedSourceStep( Block header_, const std::vector> & sources_, @@ -13,6 +43,7 @@ DistributedSourceStep::DistributedSourceStep( int parent_stage_id_, const String & node_id_, bool add_aggregation_info_, + bool add_totals_, ContextPtr context_) : ISourceStep(DataStream{.header = std::move(header_)}) , header(output_stream->header) @@ -22,17 +53,19 @@ DistributedSourceStep::DistributedSourceStep( , parent_stage_id(parent_stage_id_) , node_id(node_id_) , add_aggregation_info(add_aggregation_info_) + , add_totals(add_totals_) , context(std::move(context_)) , log(&Poco::Logger::get("DistributedSourceStep(" + query_id + "/" + toString(stage_id) + "/" + node_id + ")")) { } - void DistributedSourceStep::addPipe(Pipes & pipes, const std::shared_ptr & source) { auto distributed_source_executor = std::make_shared(header, source, query_id, node_id, parent_stage_id); - pipes.emplace_back(createDistributedSourcePipe(distributed_source_executor, add_aggregation_info, false)); + pipes.emplace_back(createDistributedSourcePipe( + distributed_source_executor, add_aggregation_info, add_totals, context->getSettingsRef().extremes, false)); pipes.back().addInterpreterContext(context); + addConvertingActions(pipes.back(), output_stream->header); } void DistributedSourceStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) diff --git a/src/Processors/QueryPlan/DistributedSourceStep.h b/src/Processors/QueryPlan/DistributedSourceStep.h index 0052d34e3fb4..2fa3743ce589 100644 --- a/src/Processors/QueryPlan/DistributedSourceStep.h +++ b/src/Processors/QueryPlan/DistributedSourceStep.h @@ -18,6 +18,7 @@ class DistributedSourceStep final : public ISourceStep parent_stage_id_, "", false, + false, context_) { } @@ -30,6 +31,7 @@ class DistributedSourceStep final : public ISourceStep int parent_stage_id_, const String & node_id_, bool add_aggregation_info_, + bool add_totals_, /// Never use, should always be false. ContextPtr context_); String getName() const override { return "DistributedSourceStep(" + toString(stage_id) + " <= " + toString(parent_stage_id) + ")"; } @@ -47,6 +49,7 @@ class DistributedSourceStep final : public ISourceStep int parent_stage_id; String node_id; bool add_aggregation_info; + bool add_totals; ContextPtr context; Poco::Logger * log; }; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index e0045f223005..0a5f36e0ab54 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -66,7 +66,7 @@ const DataStream & QueryPlan::getCurrentDataStream() const return root->step->getOutputStream(); } -void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vector> plans, ContextPtr context) +void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vector> plans, InterpreterParamsPtr interpreter_params) { if (isInitialized()) throw Exception("Cannot unite plans because current QueryPlan is already initialized", @@ -96,7 +96,7 @@ void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vectornodes)); - nodes.emplace_back(Node{.context = std::move(context), .step = std::move(step)}); + nodes.emplace_back(Node{.step = std::move(step), .interpreter_params = std::move(interpreter_params)}); root = &nodes.back(); for (auto & plan : plans) @@ -113,7 +113,7 @@ void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vectorgetName() + " to QueryPlan because " "step has no inputs, but QueryPlan is already initialized", ErrorCodes::LOGICAL_ERROR); - LOG_DEBUG(log, "Add step {} with context {}\n", step->getName(), static_cast(context.get())); - nodes.emplace_back(Node{.context = std::move(context), .step = std::move(step)}); + LOG_DEBUG(log, "Add step {} with context {}\n", step->getName(), interpreter_params ? static_cast(interpreter_params->context.get()): nullptr); + nodes.emplace_back(Node{.step = std::move(step), .interpreter_params = std::move(interpreter_params)}); root = &nodes.back(); return; } @@ -144,7 +144,7 @@ void QueryPlan::addStep(QueryPlanStepPtr step, ContextPtr context) "root header: " + root_header.dumpStructure() + "step header: " + step_header.dumpStructure(), ErrorCodes::LOGICAL_ERROR); - nodes.emplace_back(Node{.context = std::move(context), .step = std::move(step), .children = {root}}); + nodes.emplace_back(Node{.step = std::move(step), .children = {root}, .interpreter_params = std::move(interpreter_params)}); root->parent = &nodes.back(); root = root->parent; return; @@ -287,17 +287,18 @@ void QueryPlan::buildStages(ContextPtr context) { frame.node->num_parent_stages += last_node->num_parent_stages; frame.node->num_leaf_nodes_in_stage += last_node->num_leaf_nodes_in_stage; - /// If relationships between current node and all of its child nodes are shuffle, current node will have no context. - if (!frame.node->context && last_node->context) - { - frame.node->context = last_node->context; - LOG_DEBUG( - log, - "Set context({} <= {}) to {}", - frame.node->step->getName(), - last_node->step->getName(), - static_cast(frame.node->context.get())); - } + } + + /// Transfer interpreter params bottom-up. + if (!frame.node->interpreter_params && last_node->interpreter_params) + { + frame.node->interpreter_params = last_node->interpreter_params; + LOG_DEBUG( + log, + "Set context({} <= {}) to {}", + frame.node->step->getName(), + last_node->step->getName(), + static_cast(frame.node->interpreter_params->context.get())); } ++frame.visited_children; @@ -638,6 +639,7 @@ bool QueryPlan::scheduleStages(ContextMutablePtr context) parent_stage->id, *result_stage->workers.front(), false, + false, context); addStep(std::move(distributed_source_step)); } @@ -673,19 +675,20 @@ bool QueryPlan::scheduleStages(ContextMutablePtr context) /// Fill external tables(reference from Connection.cpp: void Connection::sendExternalTablesData(ExternalTablesData & data)): if (stage.is_leaf_stage) { - if (!stage.root_node->context) - LOG_DEBUG(log, "No need to prepare external tables data, because context is null."); + if (!stage.root_node->interpreter_params) + LOG_DEBUG(log, "No need to prepare external tables data, because interpreter_params is null."); else { + assert(stage.root_node->interpreter_params->context); /// 1.Construct ExternalTablesData. ExternalTablesData external_tables_data; { - const auto & external_tables = stage.root_node->context->getExternalTables(); + const auto & external_tables = stage.root_node->interpreter_params->context->getExternalTables(); LOG_DEBUG( log, "Prepare {} external tables using context {}.", external_tables.size(), - static_cast(stage.root_node->context.get())); + static_cast(stage.root_node->interpreter_params->context.get())); for (const auto & table : external_tables) { StoragePtr cur = table.second; @@ -703,13 +706,13 @@ bool QueryPlan::scheduleStages(ContextMutablePtr context) SelectQueryInfo select_query_info; auto metadata_snapshot = cur->getInMemoryMetadataPtr(); QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage( - stage.root_node->context, QueryProcessingStage::Complete, metadata_snapshot, select_query_info); + stage.root_node->interpreter_params->context, QueryProcessingStage::Complete, metadata_snapshot, select_query_info); Pipe pipe = cur->read( metadata_snapshot->getColumns().getNamesOfPhysical(), metadata_snapshot, select_query_info, - stage.root_node->context, + stage.root_node->interpreter_params->context, read_from_table_stage, DEFAULT_BLOCK_SIZE, 1); @@ -931,7 +934,8 @@ void QueryPlan::buildPlanFragment(ContextPtr context) assert(header); const auto & sources = it->second; auto distributed_source_step = std::make_unique( - header, sources, query_distributed_plan_info.initial_query_id, my_stage_id, stage_id, my_replica, add_agg_info, context); + header, sources, query_distributed_plan_info.initial_query_id, my_stage_id, stage_id, my_replica, + add_agg_info, false, context); Node * new_node = nullptr; addStep(std::move(distributed_source_step), "", new_node); distributed_source_nodes.emplace_back(new_node); /// For debug @@ -942,7 +946,11 @@ void QueryPlan::buildPlanFragment(ContextPtr context) if (result.child_aggregating_step) { const auto & settings = context->getSettingsRef(); - auto transform_params = std::make_shared(aggregating_step->getParams(), true); + bool aggregate_final = !frame.node->interpreter_params->group_by_with_totals + && !frame.node->interpreter_params->group_by_with_rollup + && !frame.node->interpreter_params->group_by_with_cube; + LOG_DEBUG(log, "MergingAggregatedStep final: {}", aggregate_final); + auto transform_params = std::make_shared(aggregating_step->getParams(), aggregate_final); transform_params->params.intermediate_header = new_node->step->getOutputStream().header; auto merging_aggregated = std::make_unique( diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 433d8c2ee0a4..9000e160e9c8 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -42,6 +43,22 @@ namespace JSONBuilder using ItemPtr = std::unique_ptr; } +/// TODO: Fields of InterpreterContext will be used to create logical operator in buildQueryPlan(). +struct InterpreterParams +{ + InterpreterParams(const ContextPtr & context_, const ASTSelectQuery & query_ast_) : context(context_) + { + group_by_with_totals = query_ast_.group_by_with_totals; + group_by_with_rollup = query_ast_.group_by_with_rollup; + group_by_with_cube = query_ast_.group_by_with_cube; + } + ContextPtr context; + bool group_by_with_totals; + bool group_by_with_rollup; + bool group_by_with_cube; +}; +using InterpreterParamsPtr = std::shared_ptr; + /// A tree of query steps. /// The goal of QueryPlan is to build QueryPipeline. /// QueryPlan let delay pipeline creation which is helpful for pipeline-level optimizations. @@ -53,8 +70,8 @@ class QueryPlan QueryPlan(QueryPlan &&); QueryPlan & operator=(QueryPlan &&); - void unitePlans(QueryPlanStepPtr step, std::vector plans, ContextPtr context = {}); - void addStep(QueryPlanStepPtr step, ContextPtr context = {}); + void unitePlans(QueryPlanStepPtr step, std::vector plans, InterpreterParamsPtr interpreter_params = {}); + void addStep(QueryPlanStepPtr step, InterpreterParamsPtr interpreter_params = {}); bool isInitialized() const { return root != nullptr; } /// Tree is not empty bool isCompleted() const; /// Tree is not empty and root hasOutputStream() @@ -113,12 +130,12 @@ class QueryPlan /// Tree node. Step and it's children. struct Node { - ContextPtr context; QueryPlanStepPtr step; std::vector children = {}; Node * parent = nullptr; int num_parent_stages = 0; /// Number of parent stages whose child is the stage current node belongs to. int num_leaf_nodes_in_stage = 0; /// Number of leaf nodes(including current node and its descendant nodes) in the same stage. + InterpreterParamsPtr interpreter_params; }; using Nodes = std::list; diff --git a/src/Processors/Sources/DistributedSource.cpp b/src/Processors/Sources/DistributedSource.cpp index d159f82383e9..de2f2b37462c 100644 --- a/src/Processors/Sources/DistributedSource.cpp +++ b/src/Processors/Sources/DistributedSource.cpp @@ -95,9 +95,55 @@ void DistributedSource::onUpdatePorts() } } -Pipe createDistributedSourcePipe(DistributedSourceExecutorPtr query_executor, bool add_aggregation_info, bool async_read) +DistributedTotalsSource::DistributedTotalsSource(DistributedSourceExecutorPtr executor_) + : ISource(executor_->getHeader()) + , executor(std::move(executor_)) { - Pipe pipe(std::make_shared(query_executor, add_aggregation_info, async_read)); +} + +DistributedTotalsSource::~DistributedTotalsSource() = default; + +Chunk DistributedTotalsSource::generate() +{ + if (auto block = executor->getTotals()) + { + UInt64 num_rows = block.rows(); + return Chunk(block.getColumns(), num_rows); + } + + return {}; +} + +DistributedExtremesSource::DistributedExtremesSource(DistributedSourceExecutorPtr executor_) + : ISource(executor_->getHeader()) + , executor(std::move(executor_)) +{ +} + +DistributedExtremesSource::~DistributedExtremesSource() = default; + +Chunk DistributedExtremesSource::generate() +{ + if (auto block = executor->getExtremes()) + { + UInt64 num_rows = block.rows(); + return Chunk(block.getColumns(), num_rows); + } + + return {}; +} + +Pipe createDistributedSourcePipe( + DistributedSourceExecutorPtr executor, bool add_aggregation_info, bool /*add_totals*/, bool /*add_extremes*/, bool async_read) +{ + Pipe pipe(std::make_shared(executor, add_aggregation_info, async_read)); + +// if (add_totals) +// pipe.addTotalsSource(std::make_shared(executor)); +// +// if (add_extremes) +// pipe.addExtremesSource(std::make_shared(executor)); + return pipe; } diff --git a/src/Processors/Sources/DistributedSource.h b/src/Processors/Sources/DistributedSource.h index 091da9679b31..d267bccd07aa 100644 --- a/src/Processors/Sources/DistributedSource.h +++ b/src/Processors/Sources/DistributedSource.h @@ -33,7 +33,40 @@ class DistributedSource : public SourceWithProgress bool is_async_state = false; }; +/// Totals source from RemoteQueryExecutor. +class DistributedTotalsSource : public ISource +{ +public: + explicit DistributedTotalsSource(DistributedSourceExecutorPtr executor_); + ~DistributedTotalsSource() override; + + String getName() const override { return "DistributedTotals"; } + +protected: + Chunk generate() override; + +private: + DistributedSourceExecutorPtr executor; +}; + +/// Extremes source from RemoteQueryExecutor. +class DistributedExtremesSource : public ISource +{ +public: + explicit DistributedExtremesSource(DistributedSourceExecutorPtr executor_); + ~DistributedExtremesSource() override; + + String getName() const override { return "DistributedExtremes"; } + +protected: + Chunk generate() override; + +private: + DistributedSourceExecutorPtr executor; +}; + /// Create pipe with distributed sources. -Pipe createDistributedSourcePipe(DistributedSourceExecutorPtr query_executor, bool add_aggregation_info, bool async_read); +/// Never use add_totals and add_extremes. +Pipe createDistributedSourcePipe(DistributedSourceExecutorPtr query_executor, bool add_aggregation_info, bool add_totals, bool add_extremes, bool async_read); } diff --git a/src/QueryPipeline/DistributedSourceExecutor.cpp b/src/QueryPipeline/DistributedSourceExecutor.cpp index aa5852f3e65a..cc90519b5db2 100644 --- a/src/QueryPipeline/DistributedSourceExecutor.cpp +++ b/src/QueryPipeline/DistributedSourceExecutor.cpp @@ -35,12 +35,28 @@ Block DistributedSourceExecutor::read() try { - auto block = client.read(); - LOG_DEBUG(log, "Read block, rows: {}, columns: {}.", block.rows(), block.columns()); - if (!block) - finished = true; - - return block; + Block block; + auto message_type = client.read(block); + switch (message_type) + { + case GRPCClient::MessageType::Data: + if (!block) + finished = true; + LOG_DEBUG(log, "Read block, rows: {}, columns: {}.", block.rows(), block.columns()); + return block; + + case GRPCClient::MessageType::Totals: + totals = block; + break; + + case GRPCClient::MessageType::Extremes: + extremes = block; + break; + +// default: +// throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown GRPC block type"); + } + return {}; } catch (...) { diff --git a/src/QueryPipeline/DistributedSourceExecutor.h b/src/QueryPipeline/DistributedSourceExecutor.h index 037dca893506..630bcf8077d0 100644 --- a/src/QueryPipeline/DistributedSourceExecutor.h +++ b/src/QueryPipeline/DistributedSourceExecutor.h @@ -24,8 +24,14 @@ class DistributedSourceExecutor const Block & getHeader() const { return header; } + /// Get totals and extremes if any. + Block getTotals() { return std::move(totals); } + Block getExtremes() { return std::move(extremes); } + private: Block header; + Block totals; + Block extremes; const std::shared_ptr source; String query_id; String node_id; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 2ad5de48ecfc..ddf6fa53aad3 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -120,7 +120,9 @@ void IStorage::read( else { auto read_step = std::make_unique(std::move(pipe), getName()); - query_plan.addStep(std::move(read_step), context); + const auto & ast = query_info.query->as(); + InterpreterParamsPtr interpreter_params = std::make_shared(context, ast); + query_plan.addStep(std::move(read_step), std::move(interpreter_params)); } } From 69e3d7c7a73eb0d3c8e06f2c4380c955fcba9f8f Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Mon, 17 Jan 2022 11:16:13 +0800 Subject: [PATCH 48/84] Fix DistributedSource constructor, not set SourceWithProgress::auto_progress(default to true) --- src/Processors/Sources/DistributedSource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Sources/DistributedSource.cpp b/src/Processors/Sources/DistributedSource.cpp index de2f2b37462c..46b584e7b048 100644 --- a/src/Processors/Sources/DistributedSource.cpp +++ b/src/Processors/Sources/DistributedSource.cpp @@ -7,7 +7,7 @@ namespace DB { DistributedSource::DistributedSource(DistributedSourceExecutorPtr executor, bool add_aggregation_info_, bool async_read_) - : SourceWithProgress(executor->getHeader(), false) + : SourceWithProgress(executor->getHeader()) , query_executor(std::move(executor)) , add_aggregation_info(add_aggregation_info_) , async_read(async_read_) From 431873cd52709e7b8331740af10855db552e12c1 Mon Sep 17 00:00:00 2001 From: madianjun Date: Mon, 17 Jan 2022 15:23:29 +0800 Subject: [PATCH 49/84] Cleanup QueryInfoWrapper until all consumers are done --- src/Server/GRPCServer.cpp | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 15746c40ecf8..8a3a54631302 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1714,9 +1714,6 @@ namespace query_info_wrapper->profile_info = executor->getProfileInfo(); } query_info_wrapper->notifyFinish(); - /// Wait all consumers to finish. - query_info_wrapper->waitConsume(); - LOG_DEBUG(log, "{} producer is {}.", query_info_key, (query_info_wrapper->cancel ? "cancelled" : "done")); } } @@ -1775,6 +1772,7 @@ namespace throwIfFailedToSendResult(); } + /// Throw exception which is from producer. if (query_info_wrapper->exception.code()) throw Exception(query_info_wrapper->exception); @@ -1861,7 +1859,6 @@ namespace cancelPlanFragment(); query_info_wrapper->notifyFinish(); } - query_info_key.clear(); } io.onException(); @@ -1934,8 +1931,16 @@ namespace query_scope.reset(); query_context.reset(); session.reset(); - if (!query_info_key.empty()) + if (call_type == CALL_EXECUTE_PLAN_FRAGMENT && !query_info_key.empty()) + { + if (query_info_wrapper) + { + /// Wait all consumers to finish. + query_info_wrapper->waitConsume(); + LOG_DEBUG(log, "{} producer is {}.", query_info_key, (query_info_wrapper->cancel ? "cancelled" : "done")); + } query_info_map->erase(query_info_key); + } } void Call::readQueryInfo() From d49beaa547939912bc8a7a3c7c6ae4d50b2dec0c Mon Sep 17 00:00:00 2001 From: madianjun Date: Mon, 17 Jan 2022 16:52:39 +0800 Subject: [PATCH 50/84] Use prewhere optimization on compute node even if no data --- src/Interpreters/InterpreterSelectQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0a1418301533..ab24ace03370 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -400,7 +400,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (try_move_to_prewhere && storage && storage->supportsPrewhere() && query.where() && !query.prewhere()) { /// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable - if (const auto & column_sizes = storage->getColumnSizes(); !column_sizes.empty()) + if (const auto & column_sizes = storage->getColumnSizes(); + !column_sizes.empty() || context->getRunningMode() == Context::RunningMode::COMPUTE) { /// Extract column compressed sizes. std::unordered_map column_compressed_sizes; From a07ff615281459b472ffe9323142fe36cd0771ed Mon Sep 17 00:00:00 2001 From: Chao Ma Date: Wed, 12 Jan 2022 15:59:19 +0800 Subject: [PATCH 51/84] Add share sessions --- src/Core/Defines.h | 1 + src/Interpreters/Context.cpp | 12 ++ src/Interpreters/Context.h | 4 + src/Interpreters/InterpreterSetQuery.cpp | 16 +++ src/Interpreters/InterpreterSetQuery.h | 2 + src/Interpreters/InterpreterSetRoleQuery.cpp | 14 +++ src/Interpreters/InterpreterSetRoleQuery.h | 4 +- src/Interpreters/InterpreterUseQuery.cpp | 12 ++ src/Interpreters/InterpreterUseQuery.h | 2 + src/Interpreters/Session.cpp | 109 +++++++++++++++++++ 10 files changed, 175 insertions(+), 1 deletion(-) diff --git a/src/Core/Defines.h b/src/Core/Defines.h index b7189b1906e1..0b6b23aeb718 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -9,6 +9,7 @@ /// Placeholder, meaningless name #define CLUSTER_TYPE_ALL "all" #define DEFAULT_ZOOKEEPER_CLUSTERS_PATH "/clickhouse/clusters" +#define DEFAULT_ZOOKEEPER_SESSIONS_PATH "/clickhouse/sessions" #define STORE_NAMESPACE "shared-store" #define DBMS_DEFAULT_PORT 9000 diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9fe4826af3ad..d34c14205e27 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1186,6 +1186,12 @@ std::shared_ptr Context::getSettingsCons } +String Context::getSessionID() const +{ + auto lock = getLock(); + return session_id; +} + String Context::getCurrentDatabase() const { auto lock = getLock(); @@ -1213,6 +1219,12 @@ void Context::setCurrentDatabaseNameInGlobalContext(const String & name) current_database = name; } +void Context::setSessionID(const String & id) +{ + auto lock = getLock(); + session_id = id; +} + void Context::setCurrentDatabase(const String & name) { DatabaseCatalog::instance().assertDatabaseExists(name); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index a03c002d878d..0f260325dc55 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -200,6 +200,7 @@ class Context: public std::enable_shared_from_this std::shared_ptr initial_row_policy; String current_database; Settings settings; /// Setting for query execution. + String session_id; using ProgressCallback = std::function; ProgressCallback progress_callback; /// Callback for tracking progress of query execution. @@ -535,12 +536,15 @@ class Context: public std::enable_shared_from_this ContextPtr getInitialContext(const String & plan_fragment_id) const; bool isStandaloneMode() const; + String getSessionID() const; String getCurrentDatabase() const; String getCurrentQueryId() const { return client_info.current_query_id; } /// Id of initiating query for distributed queries; or current query id if it's not a distributed query. String getInitialQueryId() const; + void setSessionID(const String & session_id); + void setCurrentDatabase(const String & name); /// Set current_database for global context. We don't validate that database /// exists because it should be set before databases loading. diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index 1c6a4236bf60..d08370f291c6 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -2,15 +2,20 @@ #include #include +#include + namespace DB { +namespace fs = std::filesystem; BlockIO InterpreterSetQuery::execute() { const auto & ast = query_ptr->as(); getContext()->checkSettingsConstraints(ast.changes); getContext()->getSessionContext()->applySettingsChanges(ast.changes); + if (!getContext()->getSessionID().empty()) + uploadSettingChanges(ast.changes); return {}; } @@ -20,6 +25,17 @@ void InterpreterSetQuery::executeForCurrentContext() const auto & ast = query_ptr->as(); getContext()->checkSettingsConstraints(ast.changes); getContext()->applySettingsChanges(ast.changes); + if (!getContext()->getSessionID().empty()) + uploadSettingChanges(ast.changes); +} + +void InterpreterSetQuery::uploadSettingChanges(const SettingsChanges & changes) +{ + auto zookeeper = getContext()->getZooKeeper(); + auto zookeeper_path = fs::path(DEFAULT_ZOOKEEPER_SESSIONS_PATH) / getContext()->getSessionID() / "setting_changes" / ""; + zookeeper->createAncestors(zookeeper_path); + for (const SettingChange & change : changes) + zookeeper->createOrUpdate(zookeeper_path / change.name,toString(change.value), zkutil::CreateMode::Persistent); } } diff --git a/src/Interpreters/InterpreterSetQuery.h b/src/Interpreters/InterpreterSetQuery.h index 9bd497084213..ec2e15a4a1d8 100644 --- a/src/Interpreters/InterpreterSetQuery.h +++ b/src/Interpreters/InterpreterSetQuery.h @@ -27,6 +27,8 @@ class InterpreterSetQuery : public IInterpreter, WithMutableContext private: ASTPtr query_ptr; + + void uploadSettingChanges(const SettingsChanges & changes); }; } diff --git a/src/Interpreters/InterpreterSetRoleQuery.cpp b/src/Interpreters/InterpreterSetRoleQuery.cpp index 057ccd447ef1..1939b3ccc229 100644 --- a/src/Interpreters/InterpreterSetRoleQuery.cpp +++ b/src/Interpreters/InterpreterSetRoleQuery.cpp @@ -6,9 +6,12 @@ #include #include +#include +#include namespace DB { +namespace fs = std::filesystem; namespace ErrorCodes { extern const int SET_NON_GRANTED_ROLE; @@ -25,6 +28,15 @@ BlockIO InterpreterSetRoleQuery::execute() return {}; } +void InterpreterSetRoleQuery::uploadCurrentRoles(const std::vector & roles) { + auto zookeeper = getContext()->getZooKeeper(); + auto zookeeper_path = fs::path(DEFAULT_ZOOKEEPER_SESSIONS_PATH) / getContext()->getSessionID(); + Strings res; + res.reserve(roles.size()); + for (const auto & id : roles) + res.emplace_back(toString(id)); + zookeeper->createOrUpdate(zookeeper_path / "current_roles", boost::algorithm::join(res, ","), zkutil::CreateMode::Persistent); +} void InterpreterSetRoleQuery::setRole(const ASTSetRoleQuery & query) { @@ -54,6 +66,8 @@ void InterpreterSetRoleQuery::setRole(const ASTSetRoleQuery & query) } } session_context->setCurrentRoles(new_current_roles); + if (!session_context->getSessionID().empty()) + uploadCurrentRoles(new_current_roles); } } diff --git a/src/Interpreters/InterpreterSetRoleQuery.h b/src/Interpreters/InterpreterSetRoleQuery.h index 0a489f45fcde..7b3cf2c30af7 100644 --- a/src/Interpreters/InterpreterSetRoleQuery.h +++ b/src/Interpreters/InterpreterSetRoleQuery.h @@ -2,7 +2,7 @@ #include #include - +#include namespace DB { @@ -24,6 +24,8 @@ class InterpreterSetRoleQuery : public IInterpreter, WithMutableContext void setRole(const ASTSetRoleQuery & query); void setDefaultRole(const ASTSetRoleQuery & query); + void uploadCurrentRoles(const std::vector & roles); + ASTPtr query_ptr; }; diff --git a/src/Interpreters/InterpreterUseQuery.cpp b/src/Interpreters/InterpreterUseQuery.cpp index 626d2f499c7b..614b1999c2b7 100644 --- a/src/Interpreters/InterpreterUseQuery.cpp +++ b/src/Interpreters/InterpreterUseQuery.cpp @@ -4,16 +4,28 @@ #include #include +#include namespace DB { +namespace fs = std::filesystem; + BlockIO InterpreterUseQuery::execute() { const String & new_database = query_ptr->as().database; getContext()->checkAccess(AccessType::SHOW_DATABASES, new_database); getContext()->getSessionContext()->setCurrentDatabase(new_database); + if (!getContext()->getSessionID().empty()) + uploadCurrentDatabase(new_database); return {}; } +void InterpreterUseQuery::uploadCurrentDatabase(const String & new_database) +{ + auto zookeeper = getContext()->getZooKeeper(); + auto zookeeper_path = fs::path(DEFAULT_ZOOKEEPER_SESSIONS_PATH) / getContext()->getSessionID(); + zookeeper->createOrUpdate(zookeeper_path / "current_database", new_database, zkutil::CreateMode::Persistent); +} + } diff --git a/src/Interpreters/InterpreterUseQuery.h b/src/Interpreters/InterpreterUseQuery.h index d1ce57dc64af..e262ef9406a2 100644 --- a/src/Interpreters/InterpreterUseQuery.h +++ b/src/Interpreters/InterpreterUseQuery.h @@ -18,6 +18,8 @@ class InterpreterUseQuery : public IInterpreter, WithContext private: ASTPtr query_ptr; + + void uploadCurrentDatabase(const String & new_database); }; } diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 020d297a6b9d..e45f88fe5369 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -19,6 +19,7 @@ #include #include #include +#include namespace DB @@ -90,6 +91,95 @@ class NamedSessionsStorage thread.join(); } + void initializeRemoteSession (const std::shared_ptr & session) { + auto zookeeper = session->context->getZooKeeper(); + + auto zookeeper_path = fs::path(DEFAULT_ZOOKEEPER_SESSIONS_PATH) / session->key.second; + if(zookeeper->exists(zookeeper_path)) + return; + + zookeeper->createAncestors(zookeeper_path / ""); + const auto now = std::chrono::system_clock::now(); + const auto timestamp = std::chrono::duration_cast(now.time_since_epoch()).count(); + Coordination::Requests requests; + requests.emplace_back(zkutil::makeCreateRequest(zookeeper_path / "user_id", toString(session->key.first), zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(zookeeper_path / "creation_timestamp", toString(timestamp), zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(zookeeper_path / "session_timeout", toString(std::chrono::duration(session->timeout).count()), zkutil::CreateMode::Persistent)); + zookeeper->multi(requests); + } + + void applySessionContextSettings(const std::shared_ptr & session) { + session->context->setSessionID(session->key.second); + + auto zookeeper = session->context->getZooKeeper(); + auto zookeeper_path = fs::path(DEFAULT_ZOOKEEPER_SESSIONS_PATH) / session->key.second; + if(!zookeeper->exists(zookeeper_path)) + { + LOG_WARNING(log, "Session {} not found in meta service", session->key.second); + return; + } + + String current_database; + zookeeper->tryGet(zookeeper_path / "current_database", current_database); + if (!current_database.empty()) + { + session->context->setCurrentDatabase(current_database); + LOG_DEBUG(log, "Session {} apply current database {}", session->key.second, current_database); + } + + String current_roles; + zookeeper->tryGet(zookeeper_path / "current_roles", current_roles); + if (!current_roles.empty()) + { + Strings roles_string; + boost::algorithm::split(roles_string, current_roles, boost::is_any_of(",")); + std::vector roles; + roles.reserve(roles_string.size()); + for (const auto & role : roles) + roles.emplace_back(UUID(role)); + + session->context->setCurrentRoles(roles); + LOG_DEBUG(log, "Session {} apply current roles {}", session->key.second, current_roles); + } + + Strings setting_changes; + zookeeper->tryGetChildren(zookeeper_path / "setting_changes", setting_changes); + for (auto & name : setting_changes) + { + String value; + zookeeper->tryGet(zookeeper_path / "setting_changes" / name, value); + if (!value.empty()) + { + session->context->applySettingChange(SettingChange{name, value}); + LOG_DEBUG( + log, + "Session {} apply setting change {}={}", + session->key.second, + name, + value); + } + } + + String creation_timestamp; + zookeeper->tryGet(zookeeper_path / "creation_timestamp", creation_timestamp); + String session_timeout; + zookeeper->tryGet(zookeeper_path / "session_timeout", session_timeout); + if (!creation_timestamp.empty()) + { + const auto now = std::chrono::system_clock::now(); + const auto timestamp = std::chrono::duration_cast(now.time_since_epoch()).count(); + const auto elapsed = timestamp - std::stoll(creation_timestamp); + if (elapsed >= 0) + { + const auto timeout_seconds = std::stoll(session_timeout) - elapsed; + session->timeout = std::chrono::seconds(timeout_seconds); + LOG_DEBUG(log, "Session {} apply session timeout {}s", session->key.second, timeout_seconds); + } + else + throw Exception("Current server timestamp is smaller than the creation timestamp of session", ErrorCodes::LOGICAL_ERROR); + } + } + /// Find existing session or create a new. std::pair, bool> acquireSession( const ContextPtr & global_context, @@ -113,6 +203,12 @@ class NamedSessionsStorage it = sessions.insert(std::make_pair(key, std::make_shared(key, context, timeout, *this))).first; const auto & session = it->second; + if (global_context->getRunningMode() == Context::RunningMode::COMPUTE) + { + initializeRemoteSession(session); + applySessionContextSettings(session); + } + if (!thread.joinable()) thread = ThreadFromGlobalPool{&NamedSessionsStorage::cleanThread, this}; @@ -125,6 +221,12 @@ class NamedSessionsStorage if (!session.unique()) throw Exception("Session is locked by a concurrent client.", ErrorCodes::SESSION_IS_LOCKED); + + if (global_context->getRunningMode() == Context::RunningMode::COMPUTE) + { + applySessionContextSettings(session); + } + return {session, false}; } } @@ -218,7 +320,13 @@ class NamedSessionsStorage scheduleCloseSession(*session->second, lock); } else + { + auto zookeeper = session->second->context->getZooKeeper(); + auto zookeeper_path = fs::path(DEFAULT_ZOOKEEPER_SESSIONS_PATH) / session->second->context->getSessionID(); + zookeeper->tryRemoveRecursive(zookeeper_path); + LOG_DEBUG(log, "Delete session {}", session->second->context->getSessionID()); sessions.erase(session); + } } } @@ -230,6 +338,7 @@ class NamedSessionsStorage std::condition_variable cond; ThreadFromGlobalPool thread; bool quit = false; + Poco::Logger * log = &Poco::Logger::get("NamedSessionsStorage"); }; From de874bf1cf5f678885ed12d4d204c23dddd1278a Mon Sep 17 00:00:00 2001 From: madianjun Date: Mon, 17 Jan 2022 18:29:59 +0800 Subject: [PATCH 52/84] Transfer interpreter params in building plan fragment, and add interperter params when add ReadFromMergeTree step --- src/Processors/QueryPlan/QueryPlan.cpp | 22 +++++++++++++++---- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +++- 2 files changed, 21 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 0a5f36e0ab54..63efe6073ee1 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -889,6 +889,18 @@ void QueryPlan::buildPlanFragment(ContextPtr context) if (one_child_is_visited) { + /// Transfer interpreter params bottom-up. + if (!frame.node->interpreter_params && last_node->interpreter_params) + { + frame.node->interpreter_params = last_node->interpreter_params; + LOG_DEBUG( + log, + "Set context({} <= {}) to {}", + frame.node->step->getName(), + last_node->step->getName(), + static_cast(frame.node->interpreter_params->context.get())); + } + CheckShuffleResult result; checkShuffle(frame.node, last_node, result); @@ -904,15 +916,15 @@ void QueryPlan::buildPlanFragment(ContextPtr context) assert(last_node == frame.node->children[frame.visited_children]); /// Add steps between current node and child node. - auto addStep = [this, &stage_id](QueryPlanStepPtr step, const String & description, Node * & node) + auto addStep = [this, &stage_id, &frame](QueryPlanStepPtr step, const String & description, Node * & node) { LOG_DEBUG(log, "Add step: {}, parent stage: {}", step->getName(), stage_id); step->setStepDescription(description); if (!node) - nodes.emplace_back(Node{.step = std::move(step)}); + nodes.emplace_back(Node{.step = std::move(step), .interpreter_params = frame.node->interpreter_params}); else { - nodes.emplace_back(Node{.step = std::move(step), .children = {node}}); + nodes.emplace_back(Node{.step = std::move(step), .children = {node}, .interpreter_params = frame.node->interpreter_params}); node->parent = &nodes.back(); } node = &nodes.back(); @@ -945,14 +957,16 @@ void QueryPlan::buildPlanFragment(ContextPtr context) /// If parent stage has aggregate, add MergingAggregatedStep. if (result.child_aggregating_step) { - const auto & settings = context->getSettingsRef(); + assert(frame.node->interpreter_params); bool aggregate_final = !frame.node->interpreter_params->group_by_with_totals && !frame.node->interpreter_params->group_by_with_rollup && !frame.node->interpreter_params->group_by_with_cube; LOG_DEBUG(log, "MergingAggregatedStep final: {}", aggregate_final); + auto transform_params = std::make_shared(aggregating_step->getParams(), aggregate_final); transform_params->params.intermediate_header = new_node->step->getOutputStream().header; + const auto & settings = context->getSettingsRef(); auto merging_aggregated = std::make_unique( new_node->step->getOutputStream(), std::move(transform_params), diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 106bca97a385..be3af59aaa7b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1184,7 +1184,9 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( ); QueryPlanPtr plan = std::make_unique(); - plan->addStep(std::move(read_from_merge_tree)); + const auto & ast = query_info.query->as(); + InterpreterParamsPtr interpreter_params = std::make_shared(context, ast); + plan->addStep(std::move(read_from_merge_tree), std::move(interpreter_params)); return plan; } From f23a8ec0b1c432ae4f7358d7a92101b3cdfc6c3f Mon Sep 17 00:00:00 2001 From: Chao Ma Date: Tue, 18 Jan 2022 20:41:20 +0800 Subject: [PATCH 53/84] Change empty_result_for_aggregation_by_empty_set to true --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 346651a1b7e1..85ab53de3983 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -258,7 +258,7 @@ class IColumn; M(Bool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown", 0) \ M(Bool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \ M(Bool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \ - M(Bool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \ + M(Bool, empty_result_for_aggregation_by_empty_set, true, "Return empty result when aggregating without keys on empty set.", 0) \ M(Bool, empty_result_for_aggregation_by_constant_keys_on_empty_set, true, "Return empty result when aggregating by constant keys on empty set.", 0) \ M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \ M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ From 62ddd97a718073d30a2e0e949f9f31a51daa5a7a Mon Sep 17 00:00:00 2001 From: Chao Ma Date: Tue, 18 Jan 2022 15:41:43 +0800 Subject: [PATCH 54/84] Fix intermediate stage read data multi times --- src/Processors/QueryPlan/QueryPlan.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 63efe6073ee1..8362b5f29fa0 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -456,7 +456,8 @@ bool QueryPlan::scheduleStages(ContextMutablePtr context) static std::unordered_set special_storages{"HDFS", "S3", "MySQL", "Memory"}; bool is_result_stage_moved_forward = false; - auto fillStage = [&store_replicas, &compute_replicas, this, &my_replica, &is_result_stage_moved_forward](Stage * stage) + /// Fill workers of stages with compute and store replicas, stages includes leaf stage, result stage, intermediate stage. + auto fillStage = [&store_replicas, this, &my_replica, &is_result_stage_moved_forward](Stage * stage) { /// Leaf stage. if (stage->is_leaf_stage) @@ -533,8 +534,8 @@ bool QueryPlan::scheduleStages(ContextMutablePtr context) } /// Intermediate stage. - stage->workers.reserve(compute_replicas.size()); - stage->workers.insert(stage->workers.end(), compute_replicas.begin(), compute_replicas.end()); + stage->workers.reserve(1); + stage->workers.emplace_back(std::make_shared(my_replica)); }; struct Frame From f2009c8644454f552e424e6d59352ceac1b09b34 Mon Sep 17 00:00:00 2001 From: madianjun Date: Tue, 18 Jan 2022 11:09:09 +0800 Subject: [PATCH 55/84] Fix the tow use cases of StorageValues, one of them doesn't have view source --- src/Processors/QueryPlan/QueryPlan.cpp | 36 +++++++++++++++----------- src/Processors/QueryPlan/QueryPlan.h | 2 +- 2 files changed, 22 insertions(+), 16 deletions(-) diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 8362b5f29fa0..14cd478cbb36 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -476,7 +476,10 @@ bool QueryPlan::scheduleStages(ContextMutablePtr context) /// It's StorageValues. else if (leaf_node->step->getStepDescription() == "Values") { - stage->has_view_source = true; + /// StorageValues is used in: + /// 1. Trigger materalized view: has view source. + /// 2. Execute "SELECT ... FROM values(...)": has no view source. + stage->maybe_has_view_source = true; } else if (leaf_node->step->getStepDescription() == "Input") { @@ -669,8 +672,6 @@ bool QueryPlan::scheduleStages(ContextMutablePtr context) /// Fill with data related to each stage. query_info.set_query_id(context->generateQueryId()); query_info.set_stage_id(stage.id); - query_info.set_has_view_source(stage.has_view_source); - query_info.set_has_input_function(stage.has_input_function); /// TODO: Not all stages need external tables, so choose the ones that are necessary, at least for leaf stages. /// Fill external tables(reference from Connection.cpp: void Connection::sendExternalTablesData(ExternalTablesData & data)): @@ -816,22 +817,26 @@ bool QueryPlan::scheduleStages(ContextMutablePtr context) query_info.set_node_id(*worker); LOG_DEBUG(log, "Remote plan fragment:\n{}", debugRemotePlanFragment(query_info.query(), *worker, initial_query_id, &stage)); - if (stage.has_view_source || stage.has_input_function) + if (stage.maybe_has_view_source || stage.has_input_function) { const String & plan_fragment_id = query_info.initial_query_id() + "/" + toString(query_info.stage_id()) + "/" + query_info.node_id(); - if (stage.has_view_source) + if (stage.maybe_has_view_source) { const auto & view_source = context->getViewSource(); - assert(view_source); - LOG_DEBUG( - log, - "Store initial context {} for plan fragment {}, because has view source: {}({}).", - static_cast(context.get()), - plan_fragment_id, - view_source->getStorageID().getFullNameNotQuoted(), - view_source->getName()); - context->addInitialContext(plan_fragment_id, context); + if (view_source) + { + LOG_DEBUG( + log, + "Store initial context {} for plan fragment {}, because has view source: {}({}).", + static_cast(context.get()), + plan_fragment_id, + view_source->getStorageID().getFullNameNotQuoted(), + view_source->getName()); + context->addInitialContext(plan_fragment_id, context); + } + else + stage.maybe_has_view_source = false; } else { @@ -842,8 +847,9 @@ bool QueryPlan::scheduleStages(ContextMutablePtr context) plan_fragment_id); context->addInitialContext(plan_fragment_id, context->getQueryContext()); } - } + query_info.set_has_view_source(stage.maybe_has_view_source); + query_info.set_has_input_function(stage.has_input_function); GRPCClient cli(*worker); auto result = cli.executePlanFragment(query_info); diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 9000e160e9c8..a24e3205fc07 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -150,7 +150,7 @@ class QueryPlan Node * root_node; /// Current stage's root node. std::vector leaf_nodes; /// Store leaf nodes which are from right side to left side. bool is_leaf_stage = false; /// Current stage is a leaf stage if it has any leaf node reading data from storage(not from remote). - bool has_view_source = false; /// Current stage reads data to trigger materialized view. + bool maybe_has_view_source = false; /// Current stage reads data to trigger materialized view. bool has_input_function = false; }; From 2afec23680321478adca5bb1880d302a4fa03060 Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Wed, 19 Jan 2022 09:54:01 +0000 Subject: [PATCH 56/84] Execute "optimize table" on all nodes --- src/Interpreters/InterpreterOptimizeQuery.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index 64de5ee04790..6bf4dc6331ab 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -69,8 +70,21 @@ BlockIO InterpreterOptimizeQuery::execute() } } + if (auto * ptr = typeid_cast(DatabaseCatalog::instance().getDatabase(table_id.database_name).get()); + ptr && !getContext()->getClientInfo().is_replicated_database_internal) + { + const_cast(ast).database = table_id.database_name; + return ptr->tryEnqueueReplicatedDDL(query_ptr, getContext()); + } + table->optimize(query_ptr, metadata_snapshot, ast.partition, ast.final, ast.deduplicate, column_names, getContext()); + if (auto * ptr = typeid_cast(DatabaseCatalog::instance().getDatabase(table_id.database_name).get()); + ptr && getContext()->getClientInfo().is_replicated_database_internal) + { + ptr->commitDatabase(getContext()); + } + return {}; } From 81452fdb5ee3047bbdb656a096aeb38e63e21622 Mon Sep 17 00:00:00 2001 From: Chao Ma Date: Sat, 22 Jan 2022 10:39:39 +0800 Subject: [PATCH 57/84] Revert empty_result_for_aggregation_by_empty_set to false --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 85ab53de3983..346651a1b7e1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -258,7 +258,7 @@ class IColumn; M(Bool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown", 0) \ M(Bool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \ M(Bool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \ - M(Bool, empty_result_for_aggregation_by_empty_set, true, "Return empty result when aggregating without keys on empty set.", 0) \ + M(Bool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \ M(Bool, empty_result_for_aggregation_by_constant_keys_on_empty_set, true, "Return empty result when aggregating by constant keys on empty set.", 0) \ M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \ M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ From 709d153448e8dbd3f2251aced06536ce7e25421a Mon Sep 17 00:00:00 2001 From: madianjun Date: Tue, 18 Jan 2022 15:52:00 +0800 Subject: [PATCH 58/84] Improve subquery, including many bugs fixed. --- src/Interpreters/ApplyWithSubqueryVisitor.cpp | 19 + src/Interpreters/ApplyWithSubqueryVisitor.h | 16 + src/Interpreters/Context.h | 9 +- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/ExpressionAnalyzer.h | 2 + src/Interpreters/GlobalSubqueriesVisitor.h | 4 + .../IInterpreterUnionOrSelectQuery.cpp | 20 +- .../IInterpreterUnionOrSelectQuery.h | 8 + src/Interpreters/InterpreterSelectQuery.cpp | 45 +- src/Interpreters/InterpreterSelectQuery.h | 2 + .../InterpreterSelectWithUnionQuery.cpp | 64 +- .../InterpreterSelectWithUnionQuery.h | 2 + src/Interpreters/TreeRewriter.cpp | 1 + src/Interpreters/TreeRewriter.h | 2 + .../QueryPlan/DistributedPlanner.cpp | 1114 +++++++++++++++++ src/Processors/QueryPlan/DistributedPlanner.h | 81 ++ src/Processors/QueryPlan/QueryPlan.cpp | 1040 +-------------- src/Processors/QueryPlan/QueryPlan.h | 69 +- .../DistributedSourceExecutor.cpp | 2 +- src/Server/GRPCServer.cpp | 17 +- src/Storages/MergeTree/MergeTreeData.h | 2 - 21 files changed, 1376 insertions(+), 1145 deletions(-) create mode 100644 src/Processors/QueryPlan/DistributedPlanner.cpp create mode 100644 src/Processors/QueryPlan/DistributedPlanner.h diff --git a/src/Interpreters/ApplyWithSubqueryVisitor.cpp b/src/Interpreters/ApplyWithSubqueryVisitor.cpp index 11157410501c..2c32366056d0 100644 --- a/src/Interpreters/ApplyWithSubqueryVisitor.cpp +++ b/src/Interpreters/ApplyWithSubqueryVisitor.cpp @@ -105,4 +105,23 @@ void ApplyWithSubqueryVisitor::visit(ASTFunction & func, const Data & data) } } +bool ReplaceSubqueryMatcher::needChildVisit(ASTPtr &, const ASTPtr &, Data &) +{ + return true; +} + +void ReplaceSubqueryMatcher::visit(ASTPtr & ast, Data & data) +{ + if (data.done) + return; + + if (auto * t = ast->as()) + { + assert(data.query); + t->children.clear(); + t->children.push_back(data.query->clone()); + data.done = true; + } +} + } diff --git a/src/Interpreters/ApplyWithSubqueryVisitor.h b/src/Interpreters/ApplyWithSubqueryVisitor.h index 3b17de1d6910..f389504ce814 100644 --- a/src/Interpreters/ApplyWithSubqueryVisitor.h +++ b/src/Interpreters/ApplyWithSubqueryVisitor.h @@ -3,6 +3,7 @@ #include #include +#include namespace DB { @@ -31,4 +32,19 @@ class ApplyWithSubqueryVisitor static void visit(ASTFunction & func, const Data & data); }; +class ReplaceSubqueryMatcher +{ +public: + struct Data + { + ASTPtr query; + bool done = false; + }; + + static bool needChildVisit(ASTPtr & node, const ASTPtr & child, Data & data); + static void visit(ASTPtr & ast, Data & data); +}; + +using ReplaceSubqueryVisitor = InDepthNodeVisitor; + } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 0f260325dc55..48d3e6ddd7c8 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -313,7 +313,6 @@ class Context: public std::enable_shared_from_this private: bool skip_distributed_plan = false; std::optional query_plan_fragment_info; /// It has no value if current node is initial compute node. - String select_query; using SampleBlockCache = std::unordered_map; mutable SampleBlockCache sample_block_cache; @@ -523,9 +522,11 @@ class Context: public std::enable_shared_from_this bool isInitialQuery() const { return !query_plan_fragment_info; } const QueryPlanFragmentInfo & getQueryPlanFragmentInfo() const { return query_plan_fragment_info.value(); } void setQueryPlanFragmentInfo(const QueryPlanFragmentInfo & query_plan_fragment_info_) { query_plan_fragment_info = query_plan_fragment_info_; } - - const String & getSelectQuery() const { return select_query; } - void setSelectQuery(const String & select_query_) { select_query = select_query_; } + void resetQueryPlanFragmentInfo() + { + if (query_plan_fragment_info) + query_plan_fragment_info.reset(); + } StoragePtr executeTableFunction(const ASTPtr & table_expression); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index d40a92972b20..820744cbdc41 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -411,7 +411,7 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global) { if (do_global) { - GlobalSubqueriesVisitor::Data subqueries_data(getContext(), subquery_depth, isRemoteStorage(), + GlobalSubqueriesVisitor::Data subqueries_data(getContext(), subquery_depth, shouldAddExternalStorage(), external_tables, subqueries_for_sets, has_global_subqueries); GlobalSubqueriesVisitor(subqueries_data).visit(query); } diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 9be43ea48cf6..ce26cbd9dfb5 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -194,6 +194,8 @@ class ExpressionAnalyzer : protected ExpressionAnalyzerData, private boost::nonc bool isRemoteStorage() const { return syntax->is_remote_storage; } + bool shouldAddExternalStorage() const { return syntax->should_add_external_storage; } + NamesAndTypesList getColumnsAfterArrayJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns); NamesAndTypesList analyzeJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns); }; diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 70f7c0c0359d..b7eb54cb6a1d 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -114,6 +114,10 @@ class GlobalSubqueriesMatcher auto interpreter = interpretSubquery(subquery_or_table_name, getContext(), subquery_depth, {}); + /// The main purpose is to reset plan fragment info, so subquery can be executed as a initial query. + /// Because if parent query is a secondary query, the subquery will not be executed completely. + interpreter->rewriteDistributedQuery(true); + Block sample = interpreter->getSampleBlock(); NamesAndTypesList columns = sample.getNamesAndTypesList(); diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index 24f1db869de1..e718ff8ba4a0 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -13,6 +14,16 @@ void IInterpreterUnionOrSelectQuery::extendQueryLogElemImpl(QueryLogElement & el elem.query_kind = "Select"; } +void IInterpreterUnionOrSelectQuery::rewriteDistributedQuery(bool is_subquery, [[maybe_unused]] size_t tables_count) +{ + /// Reset these fields to not share them with parent query. + if (is_subquery) + { + getContext()->resetQueryPlanFragmentInfo(); + getContext()->getClientInfo().query_kind = ClientInfo::QueryKind::INITIAL_QUERY; + getContext()->getClientInfo().current_query_id = getContext()->generateQueryId(); + } +} QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline() { @@ -20,10 +31,13 @@ QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline() buildQueryPlan(query_plan); - context->setSelectQuery(queryToString(this->query_ptr)); - bool is_built = query_plan.buildDistributedPlan(context); + query_plan.checkInitialized(); + query_plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); + + DistributedPlanner planner(query_plan, context); + planner.buildDistributedPlan(); - QueryPlanOptimizationSettings optimization_settings{.optimize_plan = !is_built}; + QueryPlanOptimizationSettings optimization_settings{.optimize_plan = false}; return std::move(*query_plan.buildQueryPipeline( optimization_settings, BuildQueryPipelineSettings::fromContext(context))); } diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index 3cd63788ca4c..f4342dd6e3f6 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -14,6 +14,7 @@ class IInterpreterUnionOrSelectQuery : public IInterpreter public: IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, ContextPtr context_, const SelectQueryOptions & options_) : query_ptr(query_ptr_) + , distributed_query_ptr(query_ptr->clone()) , context(Context::createCopy(context_)) , options(options_) , max_streams(context->getSettingsRef().max_threads) @@ -30,6 +31,8 @@ class IInterpreterUnionOrSelectQuery : public IInterpreter context->getClientInfo().distributed_query = queryToString(query_ptr); } + virtual void rewriteDistributedQuery(bool is_subquery, size_t tables_count = 0); + virtual void buildQueryPlan(QueryPlan & query_plan) = 0; QueryPipelineBuilder buildQueryPipeline(); @@ -45,8 +48,13 @@ class IInterpreterUnionOrSelectQuery : public IInterpreter const ContextMutablePtr & getContext() const { return context; } + const ASTPtr & getQueryPtr() const { return query_ptr; } + + const ASTPtr & getDistributedQueryPtr() const { return distributed_query_ptr; } + protected: ASTPtr query_ptr; + ASTPtr distributed_query_ptr; ContextMutablePtr context; Block result_header; SelectQueryOptions options; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ab24ace03370..a3371d0726a9 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -63,6 +63,7 @@ #include #include #include +#include #include #include @@ -348,7 +349,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( { interpreter_subquery = joined_tables.makeLeftTableSubquery(options.subquery()); if (interpreter_subquery) + { source_header = interpreter_subquery->getSampleBlock(); + interpreter_subquery->rewriteDistributedQuery(true); + } } joined_tables.rewriteDistributedInAndJoins(query_ptr); @@ -464,7 +468,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (!context->tryResolveStorageID({"", it.first}, Context::ResolveExternal)) { context->addExternalTable(it.first, std::move(*it.second)); - LOG_DEBUG(log, "Add external table to context {}", static_cast(context.get())); + LOG_DEBUG(log, "Add external table {} to context {}", it.first, static_cast(context.get())); } } @@ -508,10 +512,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Calculate structure of the result. result_header = getSampleBlockImpl(); - /// Maybe subquery has been rewritten with "_subqueryX", so reset distributed_query. - String maybe_rewritten_query = queryToString(query_ptr); - LOG_DEBUG(log, "[{}] Rewrite from \"{}\" to \"{}\"", static_cast(context.get()), context->getClientInfo().distributed_query, maybe_rewritten_query); - context->getClientInfo().distributed_query = std::move(maybe_rewritten_query); + + distributed_query_ptr = query_ptr->clone(); + rewriteDistributedQuery(false, joined_tables.tablesCount()); }; analyze(shouldMoveToPrewhere()); @@ -579,6 +582,26 @@ InterpreterSelectQuery::InterpreterSelectQuery( sanitizeBlock(result_header, true); } +void InterpreterSelectQuery::rewriteDistributedQuery(bool is_subquery, size_t tables_count) +{ + IInterpreterUnionOrSelectQuery::rewriteDistributedQuery(is_subquery); + + if (interpreter_subquery && tables_count == 1) + { + ReplaceSubqueryVisitor::Data data{.query = interpreter_subquery->getDistributedQueryPtr()}; + ReplaceSubqueryVisitor(data).visit(distributed_query_ptr); + } + + String maybe_rewritten_query = queryToString(distributed_query_ptr); + LOG_DEBUG( + log, + "[{}] Rewrite\n\"{}\"\n=>\n\"{}\"", + static_cast(context.get()), + context->getClientInfo().distributed_query, + maybe_rewritten_query); + context->getClientInfo().distributed_query = std::move(maybe_rewritten_query); +} + void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) { executeImpl(query_plan, std::move(input_pipe)); @@ -607,9 +630,13 @@ BlockIO InterpreterSelectQuery::execute() buildQueryPlan(query_plan); - bool is_built = query_plan.buildDistributedPlan(context); + query_plan.checkInitialized(); + query_plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); + + DistributedPlanner planner(query_plan, context); + planner.buildDistributedPlan(); - QueryPlanOptimizationSettings optimization_settings{.optimize_plan = !is_built}; + QueryPlanOptimizationSettings optimization_settings{.optimize_plan = false}; res.pipeline = QueryPipelineBuilder::getPipeline(std::move(*query_plan.buildQueryPipeline( optimization_settings, BuildQueryPipelineSettings::fromContext(context)))); return res; @@ -1228,7 +1255,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

hasGlobalSubqueries() && !subqueries_for_sets.empty()) + { executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets); + } } if (expressions.second_stage || from_aggregation_stage) @@ -1909,6 +1938,8 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (query_analyzer->hasAggregation()) interpreter_subquery->ignoreWithTotals(); + + interpreter_subquery->rewriteDistributedQuery(true); } interpreter_subquery->buildQueryPlan(query_plan); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 4298cbbb7943..075365158793 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -76,6 +76,8 @@ class InterpreterSelectQuery : public IInterpreterUnionOrSelectQuery ~InterpreterSelectQuery() override; + virtual void rewriteDistributedQuery(bool is_subquery, size_t tables_count) override; + /// Execute a query. Get the stream of blocks to read. BlockIO execute() override; diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index ed5c27d2a6b3..638de4d84172 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include @@ -166,33 +167,27 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( } options.ignore_limits |= all_nested_ignore_limits; options.ignore_quota |= all_nested_ignore_quota; +} + +void InterpreterSelectWithUnionQuery::rewriteDistributedQuery(bool is_subquery, size_t) +{ + IInterpreterUnionOrSelectQuery::rewriteDistributedQuery(is_subquery); + size_t num_plans = nested_interpreters.size(); + distributed_query_ptr = query_ptr->clone(); + ASTSelectWithUnionQuery * distributed_ast = distributed_query_ptr->as(); String rewritten_query; - for (size_t query_num = 0; query_num < num_children; ++query_num) - { - rewritten_query += nested_interpreters[query_num]->getContext()->getClientInfo().distributed_query; - if (query_num < num_children - 1) - { - if (ast->union_mode == ASTSelectWithUnionQuery::Mode::Unspecified) - { - rewritten_query += " UNION "; - } - else if (ast->union_mode == ASTSelectWithUnionQuery::Mode::ALL) - { - rewritten_query += " UNION ALL "; - } - else if (ast->union_mode == ASTSelectWithUnionQuery::Mode::DISTINCT) - { - rewritten_query += " UNION DISTINCT "; - } - else - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "When rewriting SELECT with UNION, unimplemented UNION type: {}", ast->union_mode); - } - } - } - LOG_DEBUG(log, "[{}] Rewrite from \"{}\" to: \"{}\"", static_cast(context.get()), context->getClientInfo().distributed_query, rewritten_query); + for (size_t i = 0; i < num_plans; ++i) + distributed_ast->list_of_selects->children.at(i) = nested_interpreters[i]->getDistributedQueryPtr()->clone(); + + rewritten_query = queryToString(distributed_query_ptr); + LOG_DEBUG( + log, + "[{}] Rewrite\n\"{}\"\n=>\n\"{}\"", + static_cast(context.get()), + context->getClientInfo().distributed_query, + rewritten_query); context->getClientInfo().distributed_query = std::move(rewritten_query); } @@ -288,6 +283,7 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) if (num_plans == 1) { nested_interpreters.front()->buildQueryPlan(query_plan); + nested_interpreters.front()->rewriteDistributedQuery(false); } else { @@ -298,6 +294,7 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) { plans[i] = std::make_unique(); nested_interpreters[i]->buildQueryPlan(*plans[i]); + nested_interpreters[i]->rewriteDistributedQuery(false); if (!blocksHaveEqualStructure(plans[i]->getCurrentDataStream().header, result_header)) { @@ -356,20 +353,15 @@ BlockIO InterpreterSelectWithUnionQuery::execute() QueryPlan query_plan; buildQueryPlan(query_plan); - { - /// Print the original query plan for debugging distributed table. - /// TODO: This will be removed in the future. - WriteBufferFromOwnString buf; - buf << "------ DEBUG Query Plan ------\n"; - buf << "SQL: " << context->getClientInfo().distributed_query << "\n"; - QueryPlan::ExplainPlanOptions options{.header = true, .actions = true}; - query_plan.explainPlan(buf, options); - LOG_DEBUG(log, "[{}] DEBUG query plan:\n{}", static_cast(context.get()), buf.str()); - } + rewriteDistributedQuery(false); + + query_plan.checkInitialized(); + query_plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); - bool is_built = query_plan.buildDistributedPlan(context); + DistributedPlanner planner(query_plan, context); + planner.buildDistributedPlan(); - QueryPlanOptimizationSettings optimization_settings{.optimize_plan = !is_built}; + QueryPlanOptimizationSettings optimization_settings{.optimize_plan = false}; auto pipeline_builder = query_plan.buildQueryPipeline( optimization_settings, BuildQueryPipelineSettings::fromContext(context)); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index bd1f1b6dd2ab..5839e54fabb3 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -24,6 +24,8 @@ class InterpreterSelectWithUnionQuery : public IInterpreterUnionOrSelectQuery ~InterpreterSelectWithUnionQuery() override; + virtual void rewriteDistributedQuery(bool is_subquery, size_t tables_count = 0) override; + /// Builds QueryPlan for current query. virtual void buildQueryPlan(QueryPlan & query_plan) override; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index e0968b7fce42..bcb6d7a562a3 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -725,6 +725,7 @@ TreeRewriterResult::TreeRewriterResult( { collectSourceColumns(add_special); is_remote_storage = storage && storage->isRemote(); + should_add_external_storage = is_remote_storage || (storage && storage->getName().ends_with("MergeTree")); } /// Add columns from storage to source_columns list. Deduplicate resulted list. diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index 52c62cc4cec4..13005230d64b 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -70,6 +70,8 @@ struct TreeRewriterResult /// Cache isRemote() call for storage, because it may be too heavy. bool is_remote_storage = false; + bool should_add_external_storage = false; + /// Results of scalar sub queries Scalars scalars; diff --git a/src/Processors/QueryPlan/DistributedPlanner.cpp b/src/Processors/QueryPlan/DistributedPlanner.cpp new file mode 100644 index 000000000000..a99295e30813 --- /dev/null +++ b/src/Processors/QueryPlan/DistributedPlanner.cpp @@ -0,0 +1,1114 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +DistributedPlanner::DistributedPlanner(QueryPlan & query_plan_, const ContextMutablePtr & context_) + : query_plan(query_plan_) + , context(context_) + , log(&Poco::Logger::get("DistributedPlanner")) +{ +} + +void DistributedPlanner::checkShuffle(QueryPlan::Node * current_node, QueryPlan::Node * child_node, CheckShuffleResult & result) +{ + /// Cases: + /// 1. child node is aggregate step. + /// 2. current node is limit step, child node is sort step: no need shuffle. + /// 3. current node is not limit step, child node is sort step: need shuffle. + /// 4. child node is limit step: need shuffle. + result.current_union_step = dynamic_cast(current_node->step.get()); + if (result.current_union_step) + { + LOG_DEBUG(log, "Check shuffle: child node is UnionStep"); + result.is_shuffle = true; + return; + } + + result.current_join_step = dynamic_cast(current_node->step.get()); + if (result.current_join_step) + { + LOG_DEBUG(log, "Check shuffle: current node is JoinStep(0x{})", static_cast(result.current_join_step)); + assert(current_node->children.size() == 2); + /// Only broadcast right side. + if (child_node == current_node->children[1]) + result.is_shuffle = true; + return; + } + + result.child_aggregating_step = dynamic_cast(child_node->step.get()); + if (result.child_aggregating_step) + { + LOG_DEBUG(log, "Check shuffle: child node is AggregatingStep"); + result.is_shuffle = true; + return; + } + + result.child_sorting_step = dynamic_cast(child_node->step.get()); + if (result.child_sorting_step) + { + LOG_DEBUG(log, "Check shuffle: child node is SortingStep"); + result.current_limit_step = dynamic_cast(current_node->step.get()); + } + else + { + result.child_limit_step = dynamic_cast(child_node->step.get()); + if (result.child_limit_step) + LOG_DEBUG(log, "Check shuffle: child node is LimitStep"); + } + + if ((result.child_sorting_step && !result.current_limit_step) || result.child_limit_step) + result.is_shuffle = true; +} + +void DistributedPlanner::buildStages() +{ + LOG_DEBUG(log, "===> Build stages."); + + auto createStage = [this](int id, std::stack & parent_stages, QueryPlan::Node * root_node, std::stack & leaf_nodes) { + stages.emplace_back(Stage{.id = id, .root_node = root_node}); + Stage * new_stage = &stages.back(); + + if (root_node) + { + for (int i = 0; !parent_stages.empty() && i < root_node->num_parent_stages; ++i) + { + new_stage->parents.emplace_back(parent_stages.top()); + parent_stages.top()->child = new_stage; + parent_stages.pop(); + } + for (int i = 0; !leaf_nodes.empty() && i < root_node->num_leaf_nodes_in_stage; ++i) + { + new_stage->leaf_nodes.emplace_back(leaf_nodes.top()); + /// This leaf node is a data source node reading data from storage. + if (leaf_nodes.top()->children.empty()) + new_stage->is_leaf_stage = true; + leaf_nodes.pop(); + } + } + LOG_DEBUG(log, "Create stage: id: {}, has {} parent stages and {} leaf nodes.", id, new_stage->parents.size(), new_stage->leaf_nodes.size()); + return new_stage; + }; + + struct Frame + { + QueryPlan::Node * node = {}; + int visited_children = 0; /// Number of visited children + }; + + /// Used for visiting the query plan tree. + std::stack stack; + stack.push(Frame{.node = query_plan.root}); + bool one_child_is_visited = false; + + /// Used for creating stage. + int stage_id = -1; + QueryPlan::Node * last_node = nullptr; /// Used for marking the current node's child. + QueryPlan::Node * leaf_node = nullptr; + Stage * last_stage = nullptr; + std::stack parent_stages; + std::stack leaf_nodes; + + while (!stack.empty()) + { + auto & frame = stack.top(); + + if (one_child_is_visited) + { + /// This is shuffle, create a new stage for child_node. + CheckShuffleResult result; + checkShuffle(frame.node, last_node, result); + if (result.is_shuffle) + { + ++stage_id; + last_stage = createStage(stage_id, parent_stages, last_node, leaf_nodes); + + /// The new stage is parent of current node's stage. + parent_stages.push(last_stage); + frame.node->num_parent_stages += 1; + + /// After creating new stage, current node will be in another stage, so save current node as a candidate leaf node. + leaf_node = frame.node; + leaf_nodes.push(leaf_node); + frame.node->num_leaf_nodes_in_stage += 1; + } + else + { + frame.node->num_parent_stages += last_node->num_parent_stages; + frame.node->num_leaf_nodes_in_stage += last_node->num_leaf_nodes_in_stage; + } + + /// Transfer interpreter params bottom-up. + if (!frame.node->interpreter_params && last_node->interpreter_params) + { + frame.node->interpreter_params = last_node->interpreter_params; + LOG_DEBUG( + log, + "Set context({} <= {}) to {}", + frame.node->step->getName(), + last_node->step->getName(), + static_cast(frame.node->interpreter_params->context.get())); + } + + ++frame.visited_children; + one_child_is_visited = false; + } + + if (frame.node->children.empty()) + { + if (dynamic_cast(frame.node->step.get())) + throw Exception( + "Not support building distributed plan on Distributed table engine, maybe you want to set " + "enable_distributed_plan=false", + ErrorCodes::LOGICAL_ERROR); + last_stage = nullptr; + leaf_node = frame.node; + leaf_nodes.push(leaf_node); + frame.node->num_leaf_nodes_in_stage = 1; + } + + size_t next_child = frame.visited_children; + if (next_child == frame.node->children.size() + || (dynamic_cast(frame.node->step.get()) && frame.visited_children == 1)) + { + LOG_DEBUG(log, "Visit step: {}", frame.node->step->getName()); + last_node = frame.node; + one_child_is_visited = true; + stack.pop(); + } + else + stack.push(Frame{.node = frame.node->children[next_child]}); + } + + /// Currently, child_node is the root node of query plan, create stage for it. + ++stage_id; + last_stage = createStage(stage_id, parent_stages, last_node, leaf_nodes); + + /// Append result stage for converging data. + ++stage_id; + /// Create a virtual node, used in iterating stages. + parent_stages.push(last_stage); + auto step = std::make_unique(stage_id, last_stage->id, context); + query_plan.nodes.emplace_back(QueryPlan::Node{.step = std::move(step), .children = {last_node}, .num_parent_stages = 1}); + query_plan.root = &query_plan.nodes.back(); + leaf_nodes.push(query_plan.root); + query_plan.root->num_leaf_nodes_in_stage = 1; + result_stage = createStage(stage_id, parent_stages, query_plan.root, leaf_nodes); + + debugStages(); +} + +void DistributedPlanner::debugStages() +{ + WriteBufferFromOwnString buf; + for (const auto & stage : stages) + { + if (stage.is_leaf_stage) + { + buf << "stage id (leaf) : "; + } + else + { + buf << "stage id (non-leaf) : "; + } + buf << stage.id; + if (stage.child) + { + buf << " => " << stage.child->id; + } + buf.write('\n'); + + buf << "parent stages id : "; + for (const auto parent_stage : stage.parents) + { + buf << parent_stage->id << " "; + } + buf.write('\n'); + + if (stage.root_node) + { + buf << "root node : " << stage.root_node->step->getName(); + buf.write('\n'); + } + + buf << "leaf nodes :\n"; + /// Iterate reversely, because leaf node are stored right to left. + for (auto it = stage.leaf_nodes.rbegin(); it != stage.leaf_nodes.rend(); ++it) + { + buf << " " << (*it)->step->getName(); + if ((*it)->children.empty()) + { + buf << " [S]"; + if (const auto * step = dynamic_cast((*it)->step.get())) + { + const auto & storage_id = step->getStorageID(); + buf << " (" << storage_id.database_name << "." << storage_id.table_name << ")"; + } + } + buf.write('\n'); + } + + buf << "------------------------------\n"; + } + LOG_DEBUG(log, "===> Print Stages:\n{}", buf.str()); +} + +bool DistributedPlanner::scheduleStages(PlanResult & plan_result) +{ + LOG_DEBUG(log, "===> Schedule stages."); + /// Use current query id to build the plan fragment id. + const String & initial_query_id = context->getClientInfo().current_query_id; + + /// Get my replica grpc address + String my_replica = context->getMacros()->getValue("replica") + ":" + toString(context->getServerPort("grpc_port")); + + /// Retrieve all replicas. + std::unordered_map replicas = context->getClustersWatcher().getContainer(); + LOG_DEBUG(log, "Schedule stages for query id {} across {} workers.", initial_query_id, replicas.size()); + std::vector> store_replicas, compute_replicas; + for (const auto & replica : replicas) + { + const auto & replica_info = replica.second; + LOG_DEBUG( + log, + "Check worker: {} => ({}/{}/{}, {}).", + replica.first, + replica_info->type, + replica_info->group, + replica_info->name, + replica_info->address); + + if (replica_info->type == "store") + { + store_replicas.emplace_back(std::make_shared(replica_info->address)); + } + else + { + compute_replicas.emplace_back(std::make_shared(replica_info->address)); + } + } + LOG_DEBUG(log, "{} store workers, {} compute workers.", store_replicas.size(), compute_replicas.size()); + if (store_replicas.empty() || compute_replicas.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No enough store workers({}) or compute workers({}).", store_replicas.size(), compute_replicas.size()); + + static std::unordered_set system_tables{"SystemClusters", "SystemDatabases", "SystemTables", "SystemColumns", + "SystemDictionaries", "SystemDataSkippingIndices", + "SystemFunctions", "SystemFormats", "SystemTableEngines", + "SystemUsers", "SystemRoles", "SystemGrants", "SystemRoleGrants", + "SystemCurrentRoles", "SystemEnabledRoles", "SystemRowPolicies", "SystemPrivileges", + "SystemQuotas", "SystemQuotaLimits", + "SystemSettings", "SystemSettingsProfiles", "SystemSettingsProfileElements", + "SystemZooKeeper", + "SystemNumbers", "SystemOne", "SystemZeros", + "SystemContributors", "SystemLicenses", + "SystemReplicatedMergeTreeSettings", "SystemMergeTreeSettings"}; + + static std::unordered_set special_storages{"HDFS", "S3", "MySQL", "Memory"}; + + bool is_result_stage_moved_forward = false; + auto fillStage = [&](Stage * stage) + { + /// Leaf stage. + if (stage->is_leaf_stage) + { + bool is_multi_points_data_source = false; + for (const auto leaf_node : stage->leaf_nodes) + { + /// It's a data source. + if (leaf_node->children.empty()) + { + /// It's system table or special storage. + if (system_tables.contains(leaf_node->step->getStepDescription()) || + special_storages.contains(leaf_node->step->getStepDescription())) + { + } + /// It's StorageValues. + else if (leaf_node->step->getStepDescription() == "Values") + { + /// StorageValues is used in: + /// 1. Trigger materalized view: has view source. + /// 2. Execute "SELECT ... FROM values(...)": has no view source. + stage->maybe_has_view_source = true; + } + else if (leaf_node->step->getStepDescription() == "Input") + { + stage->has_input_function = true; + } + else + { + LOG_DEBUG( + log, + "Leaf node {}({}) is multi-points data source.", + leaf_node->step->getName(), + leaf_node->step->getStepDescription()); + is_multi_points_data_source = true; + break; + } + } + } + /// Fill workers. + if (is_multi_points_data_source) + { + stage->workers.reserve(store_replicas.size()); + stage->workers.insert(stage->workers.end(), store_replicas.begin(), store_replicas.end()); + LOG_DEBUG(log, "Schedule stage {} to {} workers.", stage->id, stage->workers.size()); + } + else + { + stage->workers.emplace_back(std::make_shared(my_replica)); + LOG_DEBUG(log, "Schedule stage {} to 1 worker(local).", stage->id); + } + return; + } + + /// Result stage. + if (stage == result_stage) + { + stage->workers.emplace_back(std::make_shared(my_replica)); + + /// Maybe the last stage can be eliminated. + if (stage->parents.size() == 1) + { + auto * parent = stage->parents.front(); + /// Parent stage will be scheduled on the same worker as result stage. + if (parent->workers.size() == 1 && *(parent->workers.front()) == my_replica) + { + /// Use result stage's parent as result stage. + LOG_DEBUG(log, "Move result stage {} forward to stage {}.", result_stage->id, parent->id); + assert(result_stage == &stages.back()); + result_stage = parent; + query_plan.root = parent->root_node; + stages.pop_back(); + is_result_stage_moved_forward = true; + } + } + LOG_DEBUG(log, "Schedule stage {} to 1 worker(local).", stage->id); + return; + } + + /// Intermediate stage. + stage->workers.reserve(compute_replicas.size()); + stage->workers.insert(stage->workers.end(), compute_replicas.begin(), compute_replicas.end()); + LOG_DEBUG(log, "Schedule stage {} to {} workers.", stage->id, stage->workers.size()); + }; + + struct Frame + { + Stage * stage = {}; + int visited_parents = 0; /// Number of visited parents + }; + + assert(result_stage != nullptr); + std::stack stack; + stack.push(Frame{.stage = result_stage}); + + bool one_parent_is_visited = false; + while (!stack.empty()) + { + auto & frame = stack.top(); + + if (one_parent_is_visited) + { + ++frame.visited_parents; + one_parent_is_visited = false; + } + + size_t next_parent = frame.visited_parents; + if (next_parent == frame.stage->parents.size()) + { + LOG_DEBUG(log, "Visit stage: {}", frame.stage->id); + + fillStage(frame.stage); + + one_parent_is_visited = true; + stack.pop(); + } + else + { + stack.push(Frame{.stage = frame.stage->parents[next_parent]}); + } + } + + /// Create query info. + GRPCQueryInfo query_info; + { + /// Fill with data shared among stages. + query_info.set_database(context->getCurrentDatabase()); + query_info.set_output_format("Native"); + + assert(!context->getClientInfo().distributed_query.empty()); + query_info.set_query(context->getClientInfo().distributed_query); + query_info.set_initial_query_id(initial_query_id); + + /// Fill changed settings. + for (const auto setting : context->getSettingsRef().allChanged()) + { + (*query_info.mutable_settings())[setting.getName()] = setting.getValueString(); + } + } + + /// Send query info. + LOG_DEBUG(log, "===> Send stages."); + for (auto & stage : stages) + { + /// Don't send result stage. + if (&stage == result_stage) + { + assert(!result_stage->parents.empty()); + if (is_result_stage_moved_forward) + { + Context::QueryPlanFragmentInfo query_plan_fragment_info{ + .initial_query_id = initial_query_id, + .stage_id = stage.id, + .node_id = my_replica + }; + for (const auto parent : stage.parents) + { + query_plan_fragment_info.parent_sources[parent->id] = parent->workers; + } + query_plan_fragment_info.sinks = stage.sinks; + context->setQueryPlanFragmentInfo(query_plan_fragment_info); + } + else + { + /// Clear query plan tree. + query_plan.root = nullptr; + + /// Note: There will be only one parent stage for the result stage. + for (const auto parent_stage : result_stage->parents) + { + const QueryPlan::Node * parent_stage_node = parent_stage->root_node; + const auto & header = parent_stage_node->step->getOutputStream().header; + assert(header); + LOG_DEBUG( + log, + "Take the output stream header of {}: {}, header columns: {}.", + parent_stage_node->step->getName(), + parent_stage_node->step->getStepDescription(), + header.columns()); + + auto distributed_source_step = std::make_unique( + header, + parent_stage->workers, + initial_query_id, + result_stage->id, + parent_stage->id, + *result_stage->workers.front(), + false, + false, + context); + query_plan.addStep(std::move(distributed_source_step)); + plan_result.distributed_source_nodes.emplace_back(query_plan.root); + } + } + continue; + } + + /// Fill sinks. + if (!stage.child->workers.empty()) + { + stage.sinks.reserve(stage.child->workers.size()); + stage.sinks.insert(stage.sinks.end(), stage.child->workers.begin(), stage.child->workers.end()); + } + + LOG_DEBUG(log, "Stage {} has {} workers.", stage.id, stage.workers.size()); + assert(!stage.workers.empty()); + + /// Fill with data related to each stage. + query_info.set_query_id(context->generateQueryId()); + query_info.set_stage_id(stage.id); + + /// TODO: Not all stages need external tables, so choose the ones that are necessary, at least for leaf stages. + /// Fill external tables(reference from Connection.cpp: void Connection::sendExternalTablesData(ExternalTablesData & data)): + if (stage.is_leaf_stage) + { + if (!stage.root_node->interpreter_params) + LOG_DEBUG(log, "No need to prepare external tables data, because interpreter_params is null."); + else + { + assert(stage.root_node->interpreter_params->context); + /// 1.Construct ExternalTablesData. + ExternalTablesData external_tables_data; + { + const auto & external_tables = stage.root_node->interpreter_params->context->getExternalTables(); + LOG_DEBUG( + log, + "Prepare {} external tables using context {}.", + external_tables.size(), + static_cast(stage.root_node->interpreter_params->context.get())); + for (const auto & table : external_tables) + { + StoragePtr cur = table.second; + + auto data = std::make_unique(); + data->table_name = table.first; + + LOG_DEBUG( + log, + "Prepare external table {} with {} ({}).", + data->table_name, + cur->getStorageID().getFullNameNotQuoted(), + cur->getName()); + { + SelectQueryInfo select_query_info; + auto metadata_snapshot = cur->getInMemoryMetadataPtr(); + QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage( + stage.root_node->interpreter_params->context, QueryProcessingStage::Complete, metadata_snapshot, select_query_info); + + Pipe pipe = cur->read( + metadata_snapshot->getColumns().getNamesOfPhysical(), + metadata_snapshot, + select_query_info, + stage.root_node->interpreter_params->context, + read_from_table_stage, + DEFAULT_BLOCK_SIZE, + 1); + + if (pipe.empty()) + { + data->pipe = std::make_unique( + std::make_shared(metadata_snapshot->getSampleBlock(), Chunk())); + } + else + { + data->pipe = std::make_unique(std::move(pipe)); + } + } + external_tables_data.emplace_back(std::move(data)); + } + } + + /// Fill external tables: + /// 2.Construct grpc data. + for (auto & data : external_tables_data) + { + Stopwatch watch; + clickhouse::grpc::ExternalTable external_table; + external_table.set_name(data->table_name); + external_table.set_format("Native"); + + assert(data->pipe); + + QueryPipelineBuilder pipeline_builder; + pipeline_builder.init(std::move(*data->pipe)); + data->pipe.reset(); + pipeline_builder.resize(1); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline_builder)); + + /// Fill columns name and type. + auto header = pipeline.getHeader(); + for (size_t i = 0; i < header.columns(); ++i) + { + ColumnWithTypeAndName column = header.safeGetByPosition(i); + clickhouse::grpc::NameAndType name_and_type; + name_and_type.set_name(column.name); + name_and_type.set_type(column.type->getName()); + external_table.mutable_columns()->Add(std::move(name_and_type)); + } + + /// Fill data. + std::optional write_buffer; + write_buffer.emplace(*external_table.mutable_data()); + std::shared_ptr output_format_processor = context->getOutputFormat("Native", *write_buffer, header); + output_format_processor->doWritePrefix(); + + Block block; + size_t rows = 0, bytes = 0; + auto executor = std::make_shared(pipeline); + while (executor->pull(block, 100)) + { + if (block) + { + rows += block.rows(); + bytes += block.bytes(); + output_format_processor->write(materializeBlock(block)); + } + } + output_format_processor->doWriteSuffix(); + LOG_DEBUG( + log, + "Fill external table {} with {} rows, {} bytes in {} sec.", + external_table.name(), + rows, + bytes, + watch.elapsedSeconds()); + + query_info.mutable_external_tables()->Add(std::move(external_table)); + } + } + } + + /// Fill parents id and sources. + query_info.clear_parent_sources(); + for (const auto parent : stage.parents) + { + clickhouse::grpc::MapEntry entry; + for (const auto & source : parent->workers) + entry.add_sources(*source); + (*query_info.mutable_parent_sources())[parent->id] = entry; + } + + /// Fill sinks. + query_info.clear_sinks(); + for (const auto & sink : stage.sinks) + { + query_info.add_sinks(*sink); + } + + /// Send query info to each remote worker. + for (const auto & worker : stage.workers) + { + Stopwatch watch; + query_info.set_node_id(*worker); + LOG_DEBUG(log, "Remote plan fragment:\n{}", debugRemotePlanFragment(query_info.query(), *worker, initial_query_id, &stage)); + + if (stage.maybe_has_view_source || stage.has_input_function) + { + const String & plan_fragment_id + = query_info.initial_query_id() + "/" + toString(query_info.stage_id()) + "/" + query_info.node_id(); + if (stage.maybe_has_view_source) + { + const auto & view_source = context->getViewSource(); + if (view_source) + { + LOG_DEBUG( + log, + "Store initial context {} for plan fragment {}, because has view source: {}({}).", + static_cast(context.get()), + plan_fragment_id, + view_source->getStorageID().getFullNameNotQuoted(), + view_source->getName()); + context->addInitialContext(plan_fragment_id, context); + } + else + stage.maybe_has_view_source = false; + } + else + { + LOG_DEBUG( + log, + "Store initial context {} for plan fragment {}, because has input function.", + static_cast(context->getQueryContext().get()), + plan_fragment_id); + context->addInitialContext(plan_fragment_id, context->getQueryContext()); + } + } + query_info.set_has_view_source(stage.maybe_has_view_source); + query_info.set_has_input_function(stage.has_input_function); + + GRPCClient cli(*worker); + auto result = cli.executePlanFragment(query_info); + LOG_DEBUG( + log, + "Finish sending GRPC query info in {} sec. {}", + watch.elapsedSeconds(), + result.exception().code() == 0 + ? "" + : "Exception: (code " + toString(result.exception().code()) + ") " + result.exception().display_text()); + } + } + + return is_result_stage_moved_forward; +} + +void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) +{ + const auto & query_distributed_plan_info = context->getQueryPlanFragmentInfo(); + int my_stage_id = query_distributed_plan_info.stage_id; + LOG_DEBUG( + log, + "===> Build plan fragment: {} stage {}, has {} parent stages.", + (result_stage ? (my_stage_id == result_stage->id ? "result": "non-result") : "non-result"), + my_stage_id, + query_distributed_plan_info.parent_sources.size()); + + /// Get my replica grpc address + String my_replica = context->getMacros()->getValue("replica") + ":" + toString(context->getServerPort("grpc_port")); + + struct Frame + { + QueryPlan::Node * node = {}; + int visited_children = 0; /// Number of visited children + }; + + /// Used for visiting the query plan tree. + std::stack stack; + stack.push(Frame{.node = query_plan.root}); + bool one_child_is_visited = false; + + /// Used for locating the plan fragment. + int stage_id = -1; + QueryPlan::Node * last_node = nullptr; + + while (!stack.empty()) + { + auto & frame = stack.top(); + + if (one_child_is_visited) + { + /// Transfer interpreter params bottom-up. + if (!frame.node->interpreter_params && last_node->interpreter_params) + { + frame.node->interpreter_params = last_node->interpreter_params; + LOG_DEBUG( + log, + "Set context({} <= {}) to {}", + frame.node->step->getName(), + last_node->step->getName(), + static_cast(frame.node->interpreter_params->context.get())); + } + + CheckShuffleResult result; + checkShuffle(frame.node, last_node, result); + + /// This is a shuffle dependency between current node and the last visited child. + if (result.is_shuffle) + { + ++stage_id; + + /// This is one of my parent stages. + const auto & it = query_distributed_plan_info.parent_sources.find(stage_id); + if (it != query_distributed_plan_info.parent_sources.end()) + { + assert(last_node == frame.node->children[frame.visited_children]); + + /// Add steps between current node and child node. + auto addStep = [this, &stage_id, &frame](QueryPlanStepPtr step, const String & description, QueryPlan::Node * & node) + { + LOG_DEBUG(log, "Add step: {}, parent stage: {}", step->getName(), stage_id); + step->setStepDescription(description); + if (!node) + query_plan.nodes.emplace_back(QueryPlan::Node{.step = std::move(step), .interpreter_params = frame.node->interpreter_params}); + else + { + query_plan.nodes.emplace_back(QueryPlan::Node{.step = std::move(step), .children = {node}, .interpreter_params = frame.node->interpreter_params}); + node->parent = &query_plan.nodes.back(); + } + node = &query_plan.nodes.back(); + }; + + bool add_agg_info = false; + std::unique_ptr aggregating_step; + if (result.child_aggregating_step) + { + add_agg_info = true; + /// Create AggregatingStep, and it should be non-final. + aggregating_step = std::make_unique(*result.child_aggregating_step); + } + /// The aggregating_step header will include aggregate function. + const auto & header = result.child_aggregating_step == nullptr ? last_node->step->getOutputStream().header + : aggregating_step->getOutputStream().header; + + /// Create DistributedSourceStep. + assert(header); + const auto & sources = it->second; + auto distributed_source_step = std::make_unique( + header, sources, query_distributed_plan_info.initial_query_id, my_stage_id, stage_id, my_replica, + add_agg_info, false, context); + QueryPlan::Node * new_node = nullptr; + addStep(std::move(distributed_source_step), "", new_node); + plan_result.distributed_source_nodes.emplace_back(new_node); /// For debug + + /// If current step is JoinStep or UnionStep, only add DistributedSourceStep. + + /// If parent stage has aggregate, add MergingAggregatedStep. + if (result.child_aggregating_step) + { + assert(frame.node->interpreter_params); + bool aggregate_final = !frame.node->interpreter_params->group_by_with_totals + && !frame.node->interpreter_params->group_by_with_rollup + && !frame.node->interpreter_params->group_by_with_cube; + LOG_DEBUG(log, "MergingAggregatedStep final: {}", aggregate_final); + + auto transform_params = std::make_shared(aggregating_step->getParams(), aggregate_final); + transform_params->params.intermediate_header = new_node->step->getOutputStream().header; + + const auto & settings = context->getSettingsRef(); + auto merging_aggregated = std::make_unique( + new_node->step->getOutputStream(), + std::move(transform_params), + settings.distributed_aggregation_memory_efficient, + settings.max_threads, + settings.aggregation_memory_efficient_merge_threads); + + addStep(std::move(merging_aggregated), "Merge aggregated streams for distributed AGGREGATE", new_node); + } + + /// If parent stage has order by, add SortingStep. + if (result.child_sorting_step) + { + auto merging_sorted = std::make_unique(new_node->step->getOutputStream(), *result.child_sorting_step); + addStep(std::move(merging_sorted), "Merge sorted streams for distributed ORDER BY", new_node); + } + + /// If parent stage has limit, add LimitStep. + if (result.child_limit_step) + { + assert(last_node->children.size() == 1); + const SortingStep * grandchild_sorting_step = dynamic_cast(last_node->children[0]->step.get()); + if (grandchild_sorting_step) + { + auto merging_sorted + = std::make_unique(new_node->step->getOutputStream(), *grandchild_sorting_step); + addStep(std::move(merging_sorted), "Merge sorted streams for distributed ORDER BY", new_node); + } + + auto limit = std::make_unique(new_node->step->getOutputStream(), *result.child_limit_step); + addStep(std::move(limit), "distributed LIMIT", new_node); + } + + /// Add new child node to current node. + frame.node->children[frame.visited_children] = new_node; + } + else if (stage_id == my_stage_id) + { + auto replaceStep = [this, &stage_id](QueryPlanStepPtr step, QueryPlan::Node * & node) + { + LOG_DEBUG(log, "Replace step: {}, stage: {}", step->getName(), stage_id); + node->step = std::move(step); + }; + + /// If child is AggregatingStep. + if (result.child_aggregating_step) + { + /// If NOT optimize trivial count, replace AggregatingStep with final=false. + if (!result.child_aggregating_step->getParams().optimize_trivial_count) + { + auto aggregating_step = std::make_unique(*result.child_aggregating_step); + replaceStep(std::move(aggregating_step), last_node); + } + /// If optimize trivial count, remove AggregatingStep. + else + { + LOG_DEBUG(log, "Remove step: {}, stage: {}", result.child_aggregating_step->getName(), stage_id); + last_node = last_node->children[0]; + } + } + /// If limit step is pushed down, collect (limit + offset) rows. + else if (result.child_limit_step) + result.child_limit_step->resetLimitAndOffset(); + + query_plan.root = last_node; + + return; + } + } + + ++frame.visited_children; + one_child_is_visited = false; + } + + size_t next_child = frame.visited_children; + if (next_child == frame.node->children.size() + || (dynamic_cast(frame.node->step.get()) && frame.visited_children == 1)) + { + LOG_DEBUG(log, "Visit step: {}", frame.node->step->getName()); + last_node = frame.node; + one_child_is_visited = true; + stack.pop(); + } + else + stack.push(Frame{.node = frame.node->children[next_child]}); + } + + /// Check the last stage(in fact it's the parent stage of the result stage). + ++stage_id; + if (stage_id == my_stage_id) + query_plan.root = last_node; +} + +bool DistributedPlanner::buildDistributedPlan() +{ + if (!context->getSettingsRef().enable_distributed_plan) + { + LOG_DEBUG(log, "Skip building distributed plan, because enable_distributed_plan=false."); + return false; + } + + if (context->getInitialQueryId() == "zzzzzzzz-zzzz-zzzz-zzzz-zzzzzzzzzzzz") + { + LOG_DEBUG(log, "Skip building distributed plan, because reserved initial query id is ignored."); + return false; + } + + if (context->getSkipDistributedPlan()) + { + LOG_DEBUG(log, "Skip building distributed plan, because skip_distributed_plan is true."); + return false; + } + + { + /// Print the original query plan. + WriteBufferFromOwnString buf; + buf << "------ Original Query Plan ------\n"; + buf << "SQL: " << context->getClientInfo().distributed_query << "\n"; + QueryPlan::ExplainPlanOptions options{.header = true, .actions = true}; + query_plan.explainPlan(buf, options); + LOG_DEBUG(log, "[{}] Original query plan:\n{}", static_cast(context.get()), buf.str()); + } + + /// Query hits directly on the store worker node. + if (context->isInitialQuery() && context->getRunningMode() == Context::RunningMode::STORE) + { + LOG_DEBUG(log, "Skip building distributed plan, because initial query hits directly on store worker."); + return false; + } + + std::vector> creating_set_plans; + query_plan.collectCreatingSetPlan(creating_set_plans); + + PlanResult plan_result; + if (context->isInitialQuery()) + { + buildStages(); + + bool is_result_stage_moved_forward = scheduleStages(plan_result); + if (is_result_stage_moved_forward) + buildPlanFragment(plan_result); + + if (!creating_set_plans.empty() && !dynamic_cast(query_plan.root->step.get())) + { + /// This will change query_plan.root, so save new root. + uniteCreatingSetSteps(creating_set_plans); + result_stage->root_node = query_plan.root; + } + + if (!is_result_stage_moved_forward) + { + plan_result.initial_query_id = context->getClientInfo().current_query_id; + plan_result.stage_id = result_stage->id; + plan_result.node_id = *result_stage->workers.front(); + } + else + { + const auto & query_distributed_plan_info = context->getQueryPlanFragmentInfo(); + plan_result.initial_query_id = query_distributed_plan_info.initial_query_id; + plan_result.stage_id = query_distributed_plan_info.stage_id; + plan_result.node_id = query_distributed_plan_info.node_id; + } + } + else + { + buildPlanFragment(plan_result); + + if (!creating_set_plans.empty() && !dynamic_cast(query_plan.root->step.get())) + uniteCreatingSetSteps(creating_set_plans); + + const auto & query_distributed_plan_info = context->getQueryPlanFragmentInfo(); + plan_result.initial_query_id = query_distributed_plan_info.initial_query_id; + plan_result.stage_id = query_distributed_plan_info.stage_id; + plan_result.node_id = query_distributed_plan_info.node_id; + } + + LOG_DEBUG(log, "Result plan fragment:\n{}", debugLocalPlanFragment(plan_result)); + + return true; +} + +void DistributedPlanner::uniteCreatingSetSteps(std::vector> & creating_set_plans) +{ + if (creating_set_plans.empty()) + return; + + LOG_DEBUG(log, "Unite {} CreatingSetStep", creating_set_plans.size()); + + InterpreterParamsPtr interpreter_params = query_plan.root->interpreter_params; + + DataStreams input_streams; + input_streams.emplace_back(query_plan.getCurrentDataStream()); + + std::vector> plans; + plans.emplace_back(std::make_unique(std::move(query_plan))); + query_plan = QueryPlan(); + + for (auto & creating_set_plan : creating_set_plans) + { + input_streams.emplace_back(creating_set_plan->getCurrentDataStream()); + plans.emplace_back(std::move(creating_set_plan)); + } + + auto creating_sets = std::make_unique(std::move(input_streams)); + creating_sets->setStepDescription("Create sets before main query execution"); + query_plan.unitePlans(std::move(creating_sets), std::move(plans), std::move(interpreter_params)); +} + +String DistributedPlanner::debugLocalPlanFragment(PlanResult & plan_result) +{ + WriteBufferFromOwnString buf; + buf << "------ Local Plan Fragment ------\n"; + buf << "Fragment ID: " << plan_result.initial_query_id << "/" << plan_result.stage_id << "/" << plan_result.node_id; + buf.write('\n'); + buf << "Distributed Source Nodes: " << plan_result.distributed_source_nodes.size(); + buf.write('\n'); + for (size_t i = 0; i < plan_result.distributed_source_nodes.size(); ++i) + { + const QueryPlan::Node * node = plan_result.distributed_source_nodes[i]; + auto distributed_source_step = dynamic_cast(node->step.get()); + buf << "[" << i << "]" << distributed_source_step->getName() << ", sources: "; + for (const auto & source : distributed_source_step->getSources()) + buf << *source << " "; + buf.write('\n'); + } + buf << "\nPlan Fragment:\n"; + QueryPlan::ExplainPlanOptions options{.header = true, .actions = true}; + query_plan.explainPlan(buf, options); + return buf.str(); +} + +String DistributedPlanner::debugRemotePlanFragment( + const String & query, + const String & worker, + const String & query_id, + const Stage * stage) +{ + WriteBufferFromOwnString buf; + buf << "------ Remote Plan Fragment ------\n"; + buf << "Query: " << query; + buf.write('\n'); + buf << "Worker: " << worker; + buf.write('\n'); + buf << "Fragment ID: " << query_id << "/" << stage->id << "/" << worker; + buf.write('\n'); + buf << "Sources:\n"; + for (const auto parent : stage->parents) + { + buf << " parent stage id: " << parent->id << ", sources: "; + for (const auto & source : parent->workers) + buf << *source << " "; + buf.write('\n'); + } + buf << "Sinks: "; + for (const auto & sink : stage->sinks) + { + buf << *sink << " "; + } + buf.write('\n'); + return buf.str(); +} + + +} diff --git a/src/Processors/QueryPlan/DistributedPlanner.h b/src/Processors/QueryPlan/DistributedPlanner.h new file mode 100644 index 000000000000..ebf6aa92dc47 --- /dev/null +++ b/src/Processors/QueryPlan/DistributedPlanner.h @@ -0,0 +1,81 @@ +#pragma once + +#include + +namespace DB +{ + +class DistributedPlanner { +public: + DistributedPlanner(QueryPlan & query_plan_, const ContextMutablePtr & context); + + bool buildDistributedPlan(); + +private: + struct Stage + { + int id; /// Current stage id. + std::vector parents = {}; /// Previous stages that current stage directly depends on. + Stage * child = nullptr; + std::vector> workers; /// Replicas that current stage should be executed on. + std::vector> sinks; /// Child's workers. + QueryPlan::Node * root_node; /// Current stage's root node. + std::vector leaf_nodes; /// Store leaf nodes which are from right side to left side. + bool is_leaf_stage = false; /// Current stage is a leaf stage if it has any leaf node reading data from storage(not from remote). + bool maybe_has_view_source = false; /// Current stage reads data to trigger materialized view. + bool has_input_function = false; + }; + + /// Note: do not use vector, otherwise pointers to elements in it will be invalidated when vector increases. + using Stages = std::list; + + Stages stages; + Stage * result_stage = nullptr; + + struct PlanFragmentInfo + { + PlanFragmentInfo(int stage_id_, const String & node_id_, const std::vector & sources_, const std::vector & sinks_) + : stage_id(stage_id_), node_id(node_id_), sources(sources_), sinks(sinks_) {} + int stage_id; + String node_id; /// The replica name of plan fragment receiver, used by DistributedSource. + std::vector sources; /// Point to the nodes sending data. + std::vector sinks; /// Point to the nodes receiving data. + }; + using PlanFragmentInfoPtr = std::shared_ptr; + + struct CheckShuffleResult + { + bool is_shuffle = false; + UnionStep * current_union_step = nullptr; + JoinStep * current_join_step = nullptr; + AggregatingStep * child_aggregating_step = nullptr; + SortingStep * child_sorting_step = nullptr; + LimitStep * current_limit_step = nullptr; + LimitStep * child_limit_step = nullptr; + }; + void checkShuffle(QueryPlan::Node * current_node, QueryPlan::Node * child_node, CheckShuffleResult & result); + + struct PlanResult + { + String initial_query_id; + int stage_id; + String node_id; + std::vector distributed_source_nodes; + }; + String debugLocalPlanFragment(PlanResult & plan_result); + String debugRemotePlanFragment(const String & query, const String & receiver, const String & query_id, const Stage * stage); + + void buildStages(); + void debugStages(); + /// Return true if result stage is moved forward. + bool scheduleStages(PlanResult & plan_result); + void buildPlanFragment(PlanResult & plan_result); + void uniteCreatingSetSteps(std::vector> & creating_set_plans); + +private: + QueryPlan & query_plan; + ContextMutablePtr context; + Poco::Logger * log; +}; + +} diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 14cd478cbb36..45156b4c3e7c 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -32,7 +33,7 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; +extern const int LOGICAL_ERROR; } QueryPlan::QueryPlan() : log(&Poco::Logger::get("QueryPlan")) @@ -79,7 +80,7 @@ void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vectorgetName() + " because step has different number of inputs. " "Has " + std::to_string(plans.size()) + " plans " - "and " + std::to_string(num_inputs) + " inputs", ErrorCodes::LOGICAL_ERROR); + "and " + std::to_string(num_inputs) + " inputs", ErrorCodes::LOGICAL_ERROR); } for (size_t i = 0; i < num_inputs; ++i) @@ -88,8 +89,8 @@ void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vectorgetCurrentDataStream().header; if (!blocksHaveEqualStructure(step_header, plan_header)) throw Exception("Cannot unite QueryPlans using " + step->getName() + " because " - "it has incompatible header with plan " + root->step->getName() + " " - "plan header: " + plan_header.dumpStructure() + + "it has incompatible header with plan " + root->step->getName() + " " + "plan header: " + plan_header.dumpStructure() + "step header: " + step_header.dumpStructure(), ErrorCodes::LOGICAL_ERROR); } @@ -123,8 +124,8 @@ void QueryPlan::addStep(QueryPlanStepPtr step, InterpreterParamsPtr interpreter_ { if (isInitialized()) throw Exception("Cannot add step " + step->getName() + " to QueryPlan because " - "step has no inputs, but QueryPlan is already initialized", ErrorCodes::LOGICAL_ERROR); - LOG_DEBUG(log, "Add step {} with context {}\n", step->getName(), interpreter_params ? static_cast(interpreter_params->context.get()): nullptr); + "step has no inputs, but QueryPlan is already initialized", ErrorCodes::LOGICAL_ERROR); + LOG_DEBUG(log, "Add step {} with context {}", step->getName(), interpreter_params ? static_cast(interpreter_params->context.get()): nullptr); nodes.emplace_back(Node{.step = std::move(step), .interpreter_params = std::move(interpreter_params)}); root = &nodes.back(); return; @@ -134,14 +135,14 @@ void QueryPlan::addStep(QueryPlanStepPtr step, InterpreterParamsPtr interpreter_ { if (!isInitialized()) throw Exception("Cannot add step " + step->getName() + " to QueryPlan because " - "step has input, but QueryPlan is not initialized", ErrorCodes::LOGICAL_ERROR); + "step has input, but QueryPlan is not initialized", ErrorCodes::LOGICAL_ERROR); const auto & root_header = root->step->getOutputStream().header; const auto & step_header = step->getInputStreams().front().header; if (!blocksHaveEqualStructure(root_header, step_header)) throw Exception("Cannot add step " + step->getName() + " to QueryPlan because " - "it has incompatible header with root step " + root->step->getName() + " " - "root header: " + root_header.dumpStructure() + + "it has incompatible header with root step " + root->step->getName() + " " + "root header: " + root_header.dumpStructure() + "step header: " + step_header.dumpStructure(), ErrorCodes::LOGICAL_ERROR); nodes.emplace_back(Node{.step = std::move(step), .children = {root}, .interpreter_params = std::move(interpreter_params)}); @@ -161,1026 +162,21 @@ void QueryPlan::reset() nodes.clear(); } -void QueryPlan::checkShuffle(Node * current_node, Node * child_node, CheckShuffleResult & result) +void QueryPlan::collectCreatingSetPlan(std::vector> & creating_set_plans) { - /// Cases: - /// 1. child node is aggregate step. - /// 2. current node is limit step, child node is sort step: no need shuffle. - /// 3. current node is not limit step, child node is sort step: need shuffle. - /// 4. child node is limit step: need shuffle. - result.current_union_step = dynamic_cast(current_node->step.get()); - if (result.current_union_step) + if (creating_set_plans.empty() && dynamic_cast(root->step.get())) { - LOG_DEBUG(log, "Check shuffle: child node is UnionStep"); - result.is_shuffle = true; - return; - } - - result.current_join_step = dynamic_cast(current_node->step.get()); - if (result.current_join_step) - { - LOG_DEBUG(log, "Check shuffle: current node is JoinStep(0x{})", static_cast(result.current_join_step)); - assert(current_node->children.size() == 2); - /// Only broadcast right side. - if (child_node == current_node->children[1]) - result.is_shuffle = true; - return; - } - - result.child_aggregating_step = dynamic_cast(child_node->step.get()); - if (result.child_aggregating_step) - { - LOG_DEBUG(log, "Check shuffle: child node is AggregatingStep"); - result.is_shuffle = true; - return; - } - - result.child_sorting_step = dynamic_cast(child_node->step.get()); - if (result.child_sorting_step) - { - LOG_DEBUG(log, "Check shuffle: child node is SortingStep"); - result.current_limit_step = dynamic_cast(current_node->step.get()); - } - else - { - result.child_limit_step = dynamic_cast(child_node->step.get()); - if (result.child_limit_step) - LOG_DEBUG(log, "Check shuffle: child node is LimitStep"); - } - - if ((result.child_sorting_step && !result.current_limit_step) || result.child_limit_step) - result.is_shuffle = true; -} - -void QueryPlan::buildStages(ContextPtr context) -{ - LOG_DEBUG(log, "===> Build stages."); - - auto createStage = [this](int id, std::stack & parent_stages, Node * root_node, std::stack & leaf_nodes) { - stages.emplace_back(Stage{.id = id, .root_node = root_node}); - Stage * new_stage = &stages.back(); - - if (root_node) - { - for (int i = 0; !parent_stages.empty() && i < root_node->num_parent_stages; ++i) - { - new_stage->parents.emplace_back(parent_stages.top()); - parent_stages.top()->child = new_stage; - parent_stages.pop(); - } - for (int i = 0; !leaf_nodes.empty() && i < root_node->num_leaf_nodes_in_stage; ++i) - { - new_stage->leaf_nodes.emplace_back(leaf_nodes.top()); - /// This leaf node is a data source node reading data from storage. - if (leaf_nodes.top()->children.empty()) - new_stage->is_leaf_stage = true; - leaf_nodes.pop(); - } - } - LOG_DEBUG(log, "Create stage: id: {}, has {} parent stages and {} leaf nodes.", id, new_stage->parents.size(), new_stage->leaf_nodes.size()); - return new_stage; - }; - - struct Frame - { - Node * node = {}; - int visited_children = 0; /// Number of visited children - }; - - /// Used for visiting the query plan tree. - std::stack stack; - stack.push(Frame{.node = root}); - bool one_child_is_visited = false; - - /// Used for creating stage. - int stage_id = -1; - Node * last_node = nullptr; /// Used for marking the current node's child. - Node * leaf_node = nullptr; - Stage * last_stage = nullptr; - std::stack parent_stages; - std::stack leaf_nodes; - - while (!stack.empty()) - { - auto & frame = stack.top(); - - if (one_child_is_visited) - { - /// This is shuffle, create a new stage for child_node. - CheckShuffleResult result; - checkShuffle(frame.node, last_node, result); - if (result.is_shuffle) - { - ++stage_id; - last_stage = createStage(stage_id, parent_stages, last_node, leaf_nodes); - - /// The new stage is parent of current node's stage. - parent_stages.push(last_stage); - frame.node->num_parent_stages += 1; - - /// After creating new stage, current node will be in another stage, so save current node as a candidate leaf node. - leaf_node = frame.node; - leaf_nodes.push(leaf_node); - frame.node->num_leaf_nodes_in_stage += 1; - } - else - { - frame.node->num_parent_stages += last_node->num_parent_stages; - frame.node->num_leaf_nodes_in_stage += last_node->num_leaf_nodes_in_stage; - } - - /// Transfer interpreter params bottom-up. - if (!frame.node->interpreter_params && last_node->interpreter_params) - { - frame.node->interpreter_params = last_node->interpreter_params; - LOG_DEBUG( - log, - "Set context({} <= {}) to {}", - frame.node->step->getName(), - last_node->step->getName(), - static_cast(frame.node->interpreter_params->context.get())); - } - - ++frame.visited_children; - one_child_is_visited = false; - } - - if (frame.node->children.empty()) - { - if (dynamic_cast(frame.node->step.get())) - throw Exception( - "Not support building distributed plan on Distributed table engine, maybe you want to set " - "enable_distributed_plan=false", - ErrorCodes::LOGICAL_ERROR); - last_stage = nullptr; - leaf_node = frame.node; - leaf_nodes.push(leaf_node); - frame.node->num_leaf_nodes_in_stage = 1; - } - - size_t next_child = frame.visited_children; - if (next_child == frame.node->children.size()) - { - LOG_DEBUG(log, "Visit step: {}", frame.node->step->getName()); - last_node = frame.node; - one_child_is_visited = true; - stack.pop(); - } - else - stack.push(Frame{.node = frame.node->children[next_child]}); - } - - /// Currently, child_node is the root node of query plan, create stage for it. - ++stage_id; - last_stage = createStage(stage_id, parent_stages, last_node, leaf_nodes); - - /// Append result stage for converging data. - ++stage_id; - /// Create a virtual node, used in iterating stages. - parent_stages.push(last_stage); - auto step = std::make_unique(stage_id, last_stage->id, context); - nodes.emplace_back(Node{.step = std::move(step), .children = {last_node}, .num_parent_stages = 1}); - root = &nodes.back(); - leaf_nodes.push(root); - root->num_leaf_nodes_in_stage = 1; - result_stage = createStage(stage_id, parent_stages, root, leaf_nodes); - - debugStages(); -} - -void QueryPlan::debugStages() -{ - WriteBufferFromOwnString buf; - for (const auto & stage : stages) - { - if (stage.is_leaf_stage) - { - buf << "stage id (leaf) : "; - } - else - { - buf << "stage id (non-leaf) : "; - } - buf << stage.id; - if (stage.child) - { - buf << " => " << stage.child->id; - } - buf.write('\n'); - - buf << "parent stages id : "; - for (const auto parent_stage : stage.parents) - { - buf << parent_stage->id << " "; - } - buf.write('\n'); - - if (stage.root_node) - { - buf << "root node : " << stage.root_node->step->getName(); - buf.write('\n'); - } - - buf << "leaf nodes :\n"; - /// Iterate reversely, because leaf node are stored right to left. - for (auto it = stage.leaf_nodes.rbegin(); it != stage.leaf_nodes.rend(); ++it) - { - buf << " " << (*it)->step->getName(); - if ((*it)->children.empty()) - { - buf << " [S]"; - if (const auto * step = dynamic_cast((*it)->step.get())) - { - const auto & storage_id = step->getStorageID(); - buf << " (" << storage_id.database_name << "." << storage_id.table_name << ")"; - } - } - buf.write('\n'); - } - - buf << "------------------------------\n"; - } - LOG_DEBUG(log, "===> Print Stages:\n{}", buf.str()); -} - -bool QueryPlan::scheduleStages(ContextMutablePtr context) -{ - LOG_DEBUG(log, "===> Schedule stages."); - /// Use initial query id to build the plan fragment id. - const String & initial_query_id = context->getClientInfo().current_query_id; - - /// Get my replica grpc address - String my_replica = context->getMacros()->getValue("replica") + ":" + toString(context->getServerPort("grpc_port")); - - /// Retrieve all replicas. - std::unordered_map replicas = context->getClustersWatcher().getContainer(); - LOG_DEBUG(log, "Schedule stages for query id {} across {} workers.", initial_query_id, replicas.size()); - std::vector> store_replicas, compute_replicas; - for (const auto & replica : replicas) - { - const auto & replica_info = replica.second; - LOG_DEBUG( - log, - "Check worker: {} => ({}/{}/{}, {}).", - replica.first, - replica_info->type, - replica_info->group, - replica_info->name, - replica_info->address); - - if (replica_info->type == "store") - { - store_replicas.emplace_back(std::make_shared(replica_info->address)); - } - else - { - compute_replicas.emplace_back(std::make_shared(replica_info->address)); - } - } - LOG_DEBUG(log, "{} store workers, {} compute workers.", store_replicas.size(), compute_replicas.size()); - if (store_replicas.empty() || compute_replicas.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No enough store workers({}) or compute workers({}).", store_replicas.size(), compute_replicas.size()); - - static std::unordered_set system_tables{"SystemClusters", "SystemDatabases", "SystemTables", "SystemColumns", - "SystemDictionaries", "SystemDataSkippingIndices", - "SystemFunctions", "SystemFormats", "SystemTableEngines", - "SystemUsers", "SystemRoles", "SystemGrants", "SystemRoleGrants", - "SystemCurrentRoles", "SystemEnabledRoles", "SystemRowPolicies", "SystemPrivileges", - "SystemQuotas", "SystemQuotaLimits", - "SystemSettings", "SystemSettingsProfiles", "SystemSettingsProfileElements", - "SystemZooKeeper", - "SystemNumbers", "SystemOne", "SystemZeros", - "SystemContributors", "SystemLicenses", - "SystemReplicatedMergeTreeSettings", "SystemMergeTreeSettings"}; - - static std::unordered_set special_storages{"HDFS", "S3", "MySQL", "Memory"}; - - bool is_result_stage_moved_forward = false; - /// Fill workers of stages with compute and store replicas, stages includes leaf stage, result stage, intermediate stage. - auto fillStage = [&store_replicas, this, &my_replica, &is_result_stage_moved_forward](Stage * stage) - { - /// Leaf stage. - if (stage->is_leaf_stage) - { - bool is_multi_points_data_source = false; - for (const auto leaf_node : stage->leaf_nodes) - { - /// It's a data source. - if (leaf_node->children.empty()) - { - /// It's system table or special storage. - if (system_tables.contains(leaf_node->step->getStepDescription()) || - special_storages.contains(leaf_node->step->getStepDescription())) - { - } - /// It's StorageValues. - else if (leaf_node->step->getStepDescription() == "Values") - { - /// StorageValues is used in: - /// 1. Trigger materalized view: has view source. - /// 2. Execute "SELECT ... FROM values(...)": has no view source. - stage->maybe_has_view_source = true; - } - else if (leaf_node->step->getStepDescription() == "Input") - { - stage->has_input_function = true; - } - else - { - LOG_DEBUG( - log, - "Leaf node {}({}) is multi-points data source.", - leaf_node->step->getName(), - leaf_node->step->getStepDescription()); - is_multi_points_data_source = true; - break; - } - } - } - /// Fill workers. - if (is_multi_points_data_source) - { - LOG_DEBUG(log, "Schedule to {} workers.", store_replicas.size()); - stage->workers.reserve(store_replicas.size()); - stage->workers.insert(stage->workers.end(), store_replicas.begin(), store_replicas.end()); - } - else - { - LOG_DEBUG(log, "Schedule to 1 worker."); - stage->workers.emplace_back(std::make_shared(my_replica)); - } - return; - } - - /// Result stage. - if (stage == result_stage) - { - stage->workers.emplace_back(std::make_shared(my_replica)); - - /// Maybe the last stage can be eliminated. - if (stage->parents.size() == 1) - { - auto * parent = stage->parents.front(); - /// Parent stage will be scheduled on the same worker as result stage. - if (parent->workers.size() == 1 && *(parent->workers.front()) == my_replica) - { - /// Use result stage's parent as result stage. - LOG_DEBUG(log, "Move result stage {} forward to parent stage {}.", result_stage->id, parent->id); - assert(result_stage == &stages.back()); - result_stage = parent; - root = parent->root_node; - stages.pop_back(); - is_result_stage_moved_forward = true; - } - } - return; - } - - /// Intermediate stage. - stage->workers.reserve(1); - stage->workers.emplace_back(std::make_shared(my_replica)); - }; - - struct Frame - { - Stage * stage = {}; - int visited_parents = 0; /// Number of visited parents - }; - - assert(result_stage != nullptr); - std::stack stack; - stack.push(Frame{.stage = result_stage}); - - bool one_parent_is_visited = false; - while (!stack.empty()) - { - auto & frame = stack.top(); - - if (one_parent_is_visited) - { - ++frame.visited_parents; - one_parent_is_visited = false; - } - - size_t next_parent = frame.visited_parents; - if (next_parent == frame.stage->parents.size()) - { - LOG_DEBUG(log, "Visit stage: {}", frame.stage->id); - - fillStage(frame.stage); - - one_parent_is_visited = true; - stack.pop(); - } - else - { - stack.push(Frame{.stage = frame.stage->parents[next_parent]}); - } - } - - /// Create query info. - GRPCQueryInfo query_info; - { - /// Fill with data shared among stages. - query_info.set_database(context->getCurrentDatabase()); - query_info.set_output_format("Native"); - - assert(!context->getClientInfo().distributed_query.empty()); - query_info.set_query(context->getClientInfo().distributed_query); - query_info.set_initial_query_id(initial_query_id); - - /// Fill changed settings. - for (const auto setting : context->getSettingsRef().allChanged()) - { - (*query_info.mutable_settings())[setting.getName()] = setting.getValueString(); - } - } - - /// Send query info. - LOG_DEBUG(log, "===> Send stages."); - for (auto & stage : stages) - { - /// Don't send result stage. - if (&stage == result_stage) - { - assert(!result_stage->parents.empty()); - if (is_result_stage_moved_forward) - { - Context::QueryPlanFragmentInfo query_plan_fragment_info{ - .initial_query_id = initial_query_id, - .stage_id = stage.id, - .node_id = my_replica - }; - for (const auto parent : stage.parents) - { - query_plan_fragment_info.parent_sources[parent->id] = parent->workers; - } - query_plan_fragment_info.sinks = stage.sinks; - context->setQueryPlanFragmentInfo(query_plan_fragment_info); - } - else - { - /// Clear query plan tree. - root = nullptr; - - for (const auto parent_stage : result_stage->parents) - { - const Node * parent_stage_node = parent_stage->root_node; - const auto & header = parent_stage_node->step->getOutputStream().header; - assert(header); - LOG_DEBUG( - log, - "Take the output stream header of {}: {}, header columns: {}.", - parent_stage_node->step->getName(), - parent_stage_node->step->getStepDescription(), - header.columns()); - - auto distributed_source_step = std::make_unique( - header, - parent_stage->workers, - initial_query_id, - result_stage->id, - parent_stage->id, - *result_stage->workers.front(), - false, - false, - context); - addStep(std::move(distributed_source_step)); - } - { - /// Only for debug. - LOG_DEBUG( - log, - "Result plan fragment:\n{}", - debugLocalPlanFragment( - initial_query_id, result_stage->id, *result_stage->workers.front(), std::vector{root})); - } - } - continue; - } - - /// Fill sinks. - if (!stage.child->workers.empty()) - { - stage.sinks.reserve(stage.child->workers.size()); - stage.sinks.insert(stage.sinks.end(), stage.child->workers.begin(), stage.child->workers.end()); - } - - LOG_DEBUG(log, "Stage {} has {} workers.", stage.id, stage.workers.size()); - assert(!stage.workers.empty()); - - /// Fill with data related to each stage. - query_info.set_query_id(context->generateQueryId()); - query_info.set_stage_id(stage.id); - - /// TODO: Not all stages need external tables, so choose the ones that are necessary, at least for leaf stages. - /// Fill external tables(reference from Connection.cpp: void Connection::sendExternalTablesData(ExternalTablesData & data)): - if (stage.is_leaf_stage) - { - if (!stage.root_node->interpreter_params) - LOG_DEBUG(log, "No need to prepare external tables data, because interpreter_params is null."); - else - { - assert(stage.root_node->interpreter_params->context); - /// 1.Construct ExternalTablesData. - ExternalTablesData external_tables_data; - { - const auto & external_tables = stage.root_node->interpreter_params->context->getExternalTables(); - LOG_DEBUG( - log, - "Prepare {} external tables using context {}.", - external_tables.size(), - static_cast(stage.root_node->interpreter_params->context.get())); - for (const auto & table : external_tables) - { - StoragePtr cur = table.second; - - auto data = std::make_unique(); - data->table_name = table.first; - - LOG_DEBUG( - log, - "Prepare external table {} with {} ({}).", - data->table_name, - cur->getStorageID().getFullNameNotQuoted(), - cur->getName()); - { - SelectQueryInfo select_query_info; - auto metadata_snapshot = cur->getInMemoryMetadataPtr(); - QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage( - stage.root_node->interpreter_params->context, QueryProcessingStage::Complete, metadata_snapshot, select_query_info); - - Pipe pipe = cur->read( - metadata_snapshot->getColumns().getNamesOfPhysical(), - metadata_snapshot, - select_query_info, - stage.root_node->interpreter_params->context, - read_from_table_stage, - DEFAULT_BLOCK_SIZE, - 1); - - if (pipe.empty()) - { - data->pipe = std::make_unique( - std::make_shared(metadata_snapshot->getSampleBlock(), Chunk())); - } - else - { - data->pipe = std::make_unique(std::move(pipe)); - } - } - external_tables_data.emplace_back(std::move(data)); - } - } - - /// Fill external tables: - /// 2.Construct grpc data. - for (auto & data : external_tables_data) - { - Stopwatch watch; - clickhouse::grpc::ExternalTable external_table; - external_table.set_name(data->table_name); - external_table.set_format("Native"); - - assert(data->pipe); - - QueryPipelineBuilder pipeline_builder; - pipeline_builder.init(std::move(*data->pipe)); - data->pipe.reset(); - pipeline_builder.resize(1); - auto pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline_builder)); - - /// Fill columns name and type. - auto header = pipeline.getHeader(); - for (size_t i = 0; i < header.columns(); ++i) - { - ColumnWithTypeAndName column = header.safeGetByPosition(i); - clickhouse::grpc::NameAndType name_and_type; - name_and_type.set_name(column.name); - name_and_type.set_type(column.type->getName()); - external_table.mutable_columns()->Add(std::move(name_and_type)); - } - - /// Fill data. - std::optional write_buffer; - write_buffer.emplace(*external_table.mutable_data()); - std::shared_ptr output_format_processor = context->getOutputFormat("Native", *write_buffer, header); - output_format_processor->doWritePrefix(); - - Block block; - size_t rows = 0, bytes = 0; - auto executor = std::make_shared(pipeline); - while (executor->pull(block, 100)) - { - if (block) - { - rows += block.rows(); - bytes += block.bytes(); - output_format_processor->write(materializeBlock(block)); - } - } - output_format_processor->doWriteSuffix(); - LOG_DEBUG( - log, - "Fill external table {} with {} rows, {} bytes in {} sec.", - external_table.name(), - rows, - bytes, - watch.elapsedSeconds()); - - query_info.mutable_external_tables()->Add(std::move(external_table)); - } - } - } - - /// Fill parents id and sources. - query_info.clear_parent_sources(); - for (const auto parent : stage.parents) - { - clickhouse::grpc::MapEntry entry; - for (const auto & source : parent->workers) - entry.add_sources(*source); - (*query_info.mutable_parent_sources())[parent->id] = entry; - } - - /// Fill sinks. - query_info.clear_sinks(); - for (const auto & sink : stage.sinks) - { - query_info.add_sinks(*sink); - } - - /// Send query info to each remote worker. - for (const auto & worker : stage.workers) + LOG_DEBUG(log, "Collect {} CreatingSetStep", root->children.size() - 1); + for (auto * child : root->children) { - Stopwatch watch; - query_info.set_node_id(*worker); - LOG_DEBUG(log, "Remote plan fragment:\n{}", debugRemotePlanFragment(query_info.query(), *worker, initial_query_id, &stage)); - - if (stage.maybe_has_view_source || stage.has_input_function) + if (dynamic_cast(child->step.get())) { - const String & plan_fragment_id - = query_info.initial_query_id() + "/" + toString(query_info.stage_id()) + "/" + query_info.node_id(); - if (stage.maybe_has_view_source) - { - const auto & view_source = context->getViewSource(); - if (view_source) - { - LOG_DEBUG( - log, - "Store initial context {} for plan fragment {}, because has view source: {}({}).", - static_cast(context.get()), - plan_fragment_id, - view_source->getStorageID().getFullNameNotQuoted(), - view_source->getName()); - context->addInitialContext(plan_fragment_id, context); - } - else - stage.maybe_has_view_source = false; - } - else - { - LOG_DEBUG( - log, - "Store initial context {} for plan fragment {}, because has input function.", - static_cast(context->getQueryContext().get()), - plan_fragment_id); - context->addInitialContext(plan_fragment_id, context->getQueryContext()); - } + auto plan = std::make_unique(); + plan->root = child; + creating_set_plans.emplace_back(std::move(plan)); } - query_info.set_has_view_source(stage.maybe_has_view_source); - query_info.set_has_input_function(stage.has_input_function); - - GRPCClient cli(*worker); - auto result = cli.executePlanFragment(query_info); - LOG_DEBUG(log, "Finish sending GRPC query info in {} sec. Exception: (code {}) {}", watch.elapsedSeconds(), result.exception().code(), result.exception().display_text()); } } - - return is_result_stage_moved_forward; -} - -void QueryPlan::buildPlanFragment(ContextPtr context) -{ - const auto & query_distributed_plan_info = context->getQueryPlanFragmentInfo(); - int my_stage_id = query_distributed_plan_info.stage_id; - LOG_DEBUG( - log, - "===> Build plan fragment: {} stage {}, has {} parent stages.", - (result_stage ? (my_stage_id == result_stage->id ? "result": "no-result") : "no-result"), - my_stage_id, - query_distributed_plan_info.parent_sources.size()); - - /// Get my replica grpc address - String my_replica = context->getMacros()->getValue("replica") + ":" + toString(context->getServerPort("grpc_port")); - - struct Frame - { - Node * node = {}; - int visited_children = 0; /// Number of visited children - }; - - /// Used for visiting the query plan tree. - std::stack stack; - stack.push(Frame{.node = root}); - bool one_child_is_visited = false; - - /// Used for locating the plan fragment. - int stage_id = -1; - Node * last_node = nullptr; - std::vector distributed_source_nodes; /// Only for debug - - while (!stack.empty()) - { - auto & frame = stack.top(); - - if (one_child_is_visited) - { - /// Transfer interpreter params bottom-up. - if (!frame.node->interpreter_params && last_node->interpreter_params) - { - frame.node->interpreter_params = last_node->interpreter_params; - LOG_DEBUG( - log, - "Set context({} <= {}) to {}", - frame.node->step->getName(), - last_node->step->getName(), - static_cast(frame.node->interpreter_params->context.get())); - } - - CheckShuffleResult result; - checkShuffle(frame.node, last_node, result); - - /// This is a shuffle dependency between current node and the last visited child. - if (result.is_shuffle) - { - ++stage_id; - - /// This is one of my parent stages. - const auto & it = query_distributed_plan_info.parent_sources.find(stage_id); - if (it != query_distributed_plan_info.parent_sources.end()) - { - assert(last_node == frame.node->children[frame.visited_children]); - - /// Add steps between current node and child node. - auto addStep = [this, &stage_id, &frame](QueryPlanStepPtr step, const String & description, Node * & node) - { - LOG_DEBUG(log, "Add step: {}, parent stage: {}", step->getName(), stage_id); - step->setStepDescription(description); - if (!node) - nodes.emplace_back(Node{.step = std::move(step), .interpreter_params = frame.node->interpreter_params}); - else - { - nodes.emplace_back(Node{.step = std::move(step), .children = {node}, .interpreter_params = frame.node->interpreter_params}); - node->parent = &nodes.back(); - } - node = &nodes.back(); - }; - - bool add_agg_info = false; - std::unique_ptr aggregating_step; - if (result.child_aggregating_step) - { - add_agg_info = true; - /// Create AggregatingStep, and it should be non-final. - aggregating_step = std::make_unique(*result.child_aggregating_step); - } - /// The aggregating_step header will include aggregate function. - const auto & header = result.child_aggregating_step == nullptr ? last_node->step->getOutputStream().header - : aggregating_step->getOutputStream().header; - - /// Create DistributedSourceStep. - assert(header); - const auto & sources = it->second; - auto distributed_source_step = std::make_unique( - header, sources, query_distributed_plan_info.initial_query_id, my_stage_id, stage_id, my_replica, - add_agg_info, false, context); - Node * new_node = nullptr; - addStep(std::move(distributed_source_step), "", new_node); - distributed_source_nodes.emplace_back(new_node); /// For debug - - /// If current step is JoinStep or UnionStep, only add DistributedSourceStep. - - /// If parent stage has aggregate, add MergingAggregatedStep. - if (result.child_aggregating_step) - { - assert(frame.node->interpreter_params); - bool aggregate_final = !frame.node->interpreter_params->group_by_with_totals - && !frame.node->interpreter_params->group_by_with_rollup - && !frame.node->interpreter_params->group_by_with_cube; - LOG_DEBUG(log, "MergingAggregatedStep final: {}", aggregate_final); - - auto transform_params = std::make_shared(aggregating_step->getParams(), aggregate_final); - transform_params->params.intermediate_header = new_node->step->getOutputStream().header; - - const auto & settings = context->getSettingsRef(); - auto merging_aggregated = std::make_unique( - new_node->step->getOutputStream(), - std::move(transform_params), - settings.distributed_aggregation_memory_efficient, - settings.max_threads, - settings.aggregation_memory_efficient_merge_threads); - - addStep(std::move(merging_aggregated), "Merge aggregated streams for distributed AGGREGATE", new_node); - } - - /// If parent stage has order by, add SortingStep. - if (result.child_sorting_step) - { - auto merging_sorted = std::make_unique(new_node->step->getOutputStream(), *result.child_sorting_step); - addStep(std::move(merging_sorted), "Merge sorted streams for distributed ORDER BY", new_node); - } - - /// If parent stage has limit, add LimitStep. - if (result.child_limit_step) - { - assert(last_node->children.size() == 1); - const SortingStep * grandchild_sorting_step = dynamic_cast(last_node->children[0]->step.get()); - if (grandchild_sorting_step) - { - auto merging_sorted - = std::make_unique(new_node->step->getOutputStream(), *grandchild_sorting_step); - addStep(std::move(merging_sorted), "Merge sorted streams for distributed ORDER BY", new_node); - } - - auto limit = std::make_unique(new_node->step->getOutputStream(), *result.child_limit_step); - addStep(std::move(limit), "distributed LIMIT", new_node); - } - - /// Add new child node to current node. - frame.node->children[frame.visited_children] = new_node; - } - else if (stage_id == my_stage_id) - { - auto replaceStep = [this, &stage_id](QueryPlanStepPtr step, Node * & node) - { - LOG_DEBUG(log, "Replace step: {}, stage: {}", step->getName(), stage_id); - node->step = std::move(step); - }; - - /// If child is AggregatingStep. - if (result.child_aggregating_step) - { - /// If NOT optimize trivial count, replace AggregatingStep with final=false. - if (!result.child_aggregating_step->getParams().optimize_trivial_count) - { - auto aggregating_step = std::make_unique(*result.child_aggregating_step); - replaceStep(std::move(aggregating_step), last_node); - } - /// If optimize trivial count, remove AggregatingStep. - else - { - LOG_DEBUG(log, "Remove step: {}, stage: {}", result.child_aggregating_step->getName(), stage_id); - last_node = last_node->children[0]; - } - } - /// If limit step is pushed down, collect (limit + offset) rows. - else if (result.child_limit_step) - result.child_limit_step->resetLimitAndOffset(); - - root = last_node; - { - /// Only for debug. - LOG_DEBUG( - log, - "Local plan fragment:\n{}", - debugLocalPlanFragment(query_distributed_plan_info.initial_query_id, stage_id, my_replica, distributed_source_nodes)); - } - return; - } - } - - ++frame.visited_children; - one_child_is_visited = false; - } - - size_t next_child = frame.visited_children; - if (next_child == frame.node->children.size()) - { - LOG_DEBUG(log, "Visit step: {}", frame.node->step->getName()); - last_node = frame.node; - one_child_is_visited = true; - stack.pop(); - } - else - stack.push(Frame{.node = frame.node->children[next_child]}); - } - - /// Check the last stage(in fact it's the parent stage of the result stage). - ++stage_id; - if (stage_id == my_stage_id) - { - root = last_node; - { - /// Only for debug. - LOG_DEBUG( - log, - "Local plan fragment:\n{}", - debugLocalPlanFragment(query_distributed_plan_info.initial_query_id, stage_id, my_replica, distributed_source_nodes)); - } - return; - } -} - -bool QueryPlan::buildDistributedPlan(ContextMutablePtr context) -{ - if (!context->getSettingsRef().enable_distributed_plan) - { - LOG_DEBUG(log, "Skip building distributed plan, because enable_distributed_plan=false."); - return false; - } - /// Query hits directly on the store worker node. - if (context->isInitialQuery() && context->getRunningMode() == Context::RunningMode::STORE) - { - LOG_DEBUG(log, "Skip building distributed plan, because initial query hits directly on store worker."); - return false; - } - - if (context->getInitialQueryId() == "zzzzzzzz-zzzz-zzzz-zzzz-zzzzzzzzzzzz") - { - LOG_DEBUG(log, "Skip building distributed plan, because reserved initial query id is ignored."); - return false; - } - - if (context->getSkipDistributedPlan()) - { - LOG_DEBUG(log, "Skip building distributed plan, because skip_distributed_plan is true."); - return false; - } - - { - /// Print the original query plan. - WriteBufferFromOwnString buf; - buf << "------ Original Query Plan ------\n"; - buf << "SQL: " << context->getClientInfo().distributed_query << "\n"; - QueryPlan::ExplainPlanOptions options{.header = true, .actions = true}; - explainPlan(buf, options); - LOG_DEBUG(log, "[{}] Original query plan:\n{}", static_cast(context.get()), buf.str()); - } - - checkInitialized(); - optimize(QueryPlanOptimizationSettings::fromContext(context)); - if (context->isInitialQuery()) - { - buildStages(context); - if (scheduleStages(context)) - buildPlanFragment(context); - } - else - { - buildPlanFragment(context); - } - return true; -} - -String QueryPlan::debugLocalPlanFragment(const String & query_id, int stage_id, const String & node_id, const std::vector distributed_source_nodes) -{ - WriteBufferFromOwnString buf; - buf << "------ Local Plan Fragment ------\n"; - buf << "Fragment ID: " << query_id << "/" << stage_id << "/" << node_id; - buf.write('\n'); - buf << "Distributed Source Nodes: " << distributed_source_nodes.size(); - buf.write('\n'); - for (size_t i = 0; i < distributed_source_nodes.size(); ++i) - { - const Node * node = distributed_source_nodes[i]; - auto distributed_source_step = dynamic_cast(node->step.get()); - buf << "[" << i << "]" << distributed_source_step->getName() << ", sources: "; - for (const auto & source : distributed_source_step->getSources()) - buf << *source << " "; - buf.write('\n'); - } - buf << "\nPlan Fragment:\n"; - ExplainPlanOptions options{.header = true, .actions = true}; - explainPlan(buf, options); - return buf.str(); -} - -String QueryPlan::debugRemotePlanFragment(const String & query, const String & worker, const String & query_id, const Stage * stage) -{ - WriteBufferFromOwnString buf; - buf << "------ Remote Plan Fragment ------\n"; - buf << "Query: " << query; - buf.write('\n'); - buf << "Worker: " << worker; - buf.write('\n'); - buf << "Fragment ID: " << query_id << "/" << stage->id << "/" << worker; - buf.write('\n'); - buf << "Sources:\n"; - for (const auto parent : stage->parents) - { - buf << " parent stage id: " << parent->id << ", sources: "; - for (const auto & source : parent->workers) - buf << *source << " "; - buf.write('\n'); - } - buf << "Sinks: "; - for (const auto & sink : stage->sinks) - { - buf << *sink << " "; - } - buf.write('\n'); - return buf.str(); } QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline( diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index a24e3205fc07..70d3ba8692e3 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -37,6 +37,8 @@ class LimitStep; struct QueryPlanOptimizationSettings; struct BuildQueryPipelineSettings; +class DistirbutedPlanner; + namespace JSONBuilder { class IItem; @@ -65,6 +67,7 @@ using InterpreterParamsPtr = std::shared_ptr; class QueryPlan { public: + friend class DistirbutedPlanner; QueryPlan(); ~QueryPlan(); QueryPlan(QueryPlan &&); @@ -81,12 +84,7 @@ class QueryPlan void reset(); - void buildStages(ContextPtr context); /// Used by initial node. - void debugStages(); - /// Return true if result stage is moved forward. - bool scheduleStages(ContextMutablePtr context); /// Used by initial node. - void buildPlanFragment(ContextPtr context); /// Used by non-initial nodes. - bool buildDistributedPlan(ContextMutablePtr context); + void collectCreatingSetPlan(std::vector> & creating_set_plans); QueryPipelineBuilderPtr buildQueryPipeline( const QueryPlanOptimizationSettings & optimization_settings, @@ -140,49 +138,7 @@ class QueryPlan using Nodes = std::list; - struct Stage - { - int id; /// Current stage id. - std::vector parents = {}; /// Previous stages that current stage directly depends on. - Stage * child = nullptr; - std::vector> workers; /// Replicas that current stage should be executed on. - std::vector> sinks; /// Child's workers. - Node * root_node; /// Current stage's root node. - std::vector leaf_nodes; /// Store leaf nodes which are from right side to left side. - bool is_leaf_stage = false; /// Current stage is a leaf stage if it has any leaf node reading data from storage(not from remote). - bool maybe_has_view_source = false; /// Current stage reads data to trigger materialized view. - bool has_input_function = false; - }; - - /// Note: do not use vector, otherwise pointers to elements in it will be invalidated when vector increases. - using Stages = std::list; - - struct PlanFragmentInfo - { - PlanFragmentInfo(int stage_id_, const String & node_id_, const std::vector & sources_, const std::vector & sinks_) - : stage_id(stage_id_), node_id(node_id_), sources(sources_), sinks(sinks_) {} - int stage_id; - String node_id; /// The replica name of plan fragment receiver, used by DistributedSource. - std::vector sources; /// Point to the nodes sending data. - std::vector sinks; /// Point to the nodes receiving data. - }; - using PlanFragmentInfoPtr = std::shared_ptr; - - struct CheckShuffleResult - { - bool is_shuffle = false; - UnionStep * current_union_step = nullptr; - JoinStep * current_join_step = nullptr; - AggregatingStep * child_aggregating_step = nullptr; - SortingStep * child_sorting_step = nullptr; - LimitStep * current_limit_step = nullptr; - LimitStep * child_limit_step = nullptr; - }; - void checkShuffle(Node * current_node, Node * child_node, CheckShuffleResult & result); - String debugLocalPlanFragment(const String & query_id, int stage_id, const String & node_id, const std::vector distributed_source_nodes); - String debugRemotePlanFragment(const String & query, const String & receiver, const String & query_id, const Stage * stage); - -private: +public: Nodes nodes; Node * root = nullptr; @@ -193,21 +149,6 @@ class QueryPlan size_t max_threads = 0; std::vector interpreter_context; - Stages stages; - Stage * result_stage = nullptr; - - struct pairHasher { - template - size_t operator()(const std::pair & p) const - { - auto hash1 = std::hash{}(p.first); - auto hash2 = std::hash{}(p.second); - return hash1 ^ hash2; - } - }; - - /// Key is {stage_id, receiver_address}. - std::unordered_map>, PlanFragmentInfoPtr, pairHasher> plan_fragment_infos; Poco::Logger * log; }; diff --git a/src/QueryPipeline/DistributedSourceExecutor.cpp b/src/QueryPipeline/DistributedSourceExecutor.cpp index cc90519b5db2..7d392fd09f9a 100644 --- a/src/QueryPipeline/DistributedSourceExecutor.cpp +++ b/src/QueryPipeline/DistributedSourceExecutor.cpp @@ -15,7 +15,7 @@ DistributedSourceExecutor::DistributedSourceExecutor( , node_id(node_id_) , stage_id(stage_id_) , client(*source_) - , log(&Poco::Logger::get("DistributedSourceExecutor(" + query_id + "/" + toString(stage_id) + "/" + node_id + ")")) + , log(&Poco::Logger::get("DistributedSourceExecutor(" + query_id + "/" + toString(stage_id) + "/" + *source + ")")) { GRPCTicket ticket; ticket.set_initial_query_id(query_id); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 8a3a54631302..5ba9aaf13606 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -217,15 +217,17 @@ namespace String getQueryDescription(const GRPCQueryInfo & query_info) { String str; + if (!query_info.initial_query_id().empty()) + str.append("query info key: ").append(query_info.initial_query_id() + "/" + toString(query_info.stage_id())); if (!query_info.query().empty()) { std::string_view query = query_info.query(); - constexpr size_t max_query_length_to_log = 64; + constexpr size_t max_query_length_to_log = 512; if (query.length() > max_query_length_to_log) query.remove_suffix(query.length() - max_query_length_to_log); if (size_t format_pos = query.find(" FORMAT "); format_pos != String::npos) query.remove_suffix(query.length() - format_pos - strlen(" FORMAT ")); - str.append("\"").append(query); + str.append(str.empty() ? "" : ", ").append("query: ").append("\"").append(query); if (query != query_info.query()) str.append("..."); str.append("\""); @@ -1112,7 +1114,7 @@ namespace readTicket(); - LOG_DEBUG(log, "Received ticket: {}", ticket.initial_query_id() + "/" + std::to_string(ticket.stage_id()) + "/" + ticket.node_id()); + LOG_DEBUG(log, "Received ticket: {}", ticket.initial_query_id() + "/" + std::to_string(ticket.stage_id()) + "|" + ticket.node_id()); } void Call::executeQuery() @@ -1322,6 +1324,7 @@ namespace throw Exception("Query info key " + query_info_key + " already exists", ErrorCodes::LOGICAL_ERROR); } query_info_wrapper = res.first->second; + LOG_DEBUG(log, "Store query info key {}", query_info_key); } void Call::loadQueryInfoWrapper(bool is_cancel) @@ -1329,7 +1332,10 @@ namespace query_info_key = ticket.initial_query_id() + "/" + std::to_string(ticket.stage_id()); auto res = query_info_map->get(query_info_key); if (res.second) + { query_info_wrapper = res.first; + LOG_DEBUG(log, "Load query info key {} to {}", query_info_key, ticket.node_id()); + } else { if (is_cancel) /// Plan fragment maybe done. @@ -1496,6 +1502,7 @@ namespace auto temporary_table = TemporaryTableHolder(query_context, ColumnsDescription{columns}, {}); storage = temporary_table.getTable(); query_context->addExternalTable(temporary_id.table_name, std::move(temporary_table)); + LOG_DEBUG(log, "Add external table {} to context {}", temporary_id.table_name, static_cast(query_context.get())); } if (!external_table.data().empty()) @@ -1747,7 +1754,7 @@ namespace break; block = query_info_wrapper->blocks[index]; - LOG_DEBUG(log, "{}/{} consume 1 block: {} rows, {} columns, {} bytes.", query_info_key, ticket.node_id(), block.rows(), block.columns(), block.bytes()); + LOG_DEBUG(log, "{}|{} consume 1 block: {} rows, {} columns, {} bytes.", query_info_key, ticket.node_id(), block.rows(), block.columns(), block.bytes()); query_info_wrapper->notifyProduce(); throwIfFailedToSendResult(); @@ -1786,7 +1793,7 @@ namespace output_format_processor->doWriteSuffix(); /// Notify producer that current consumer is finished. query_info_wrapper->notifyProduce(); - LOG_DEBUG(log, "{}/{} consumer is {}.", query_info_key, ticket.node_id(), (query_info_wrapper->cancel ? "cancelled" : "done")); + LOG_DEBUG(log, "{}|{} consumer is {}.", query_info_key, ticket.node_id(), (query_info_wrapper->cancel ? "cancelled" : "done")); } void Call::cancelPlanFragment() diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index bd60da15c82b..22ec7ce6f53e 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -365,8 +365,6 @@ class MergeTreeData : public IStorage, public WithMutableContext bool attach, BrokenPartCallback broken_part_callback_ = [](const String &){}); - bool isRemote() const override { return true; } - /// Build a block of minmax and count values of a MergeTree table. These values are extracted /// from minmax_indices, the first expression of primary key, and part rows. /// From 9aad8618872a90ce97e273ca380f4dfe13a4e683 Mon Sep 17 00:00:00 2001 From: madianjun Date: Sat, 22 Jan 2022 18:09:26 +0800 Subject: [PATCH 59/84] Fix friend class --- src/Processors/QueryPlan/QueryPlan.h | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 70d3ba8692e3..2c1b9cb52e9a 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -37,8 +37,6 @@ class LimitStep; struct QueryPlanOptimizationSettings; struct BuildQueryPipelineSettings; -class DistirbutedPlanner; - namespace JSONBuilder { class IItem; @@ -67,7 +65,6 @@ using InterpreterParamsPtr = std::shared_ptr; class QueryPlan { public: - friend class DistirbutedPlanner; QueryPlan(); ~QueryPlan(); QueryPlan(QueryPlan &&); @@ -138,7 +135,9 @@ class QueryPlan using Nodes = std::list; -public: +private: + friend class DistributedPlanner; + Nodes nodes; Node * root = nullptr; From 5c01196748a358fafeeadf4b721c230126c418de Mon Sep 17 00:00:00 2001 From: madianjun Date: Sun, 23 Jan 2022 10:40:17 +0800 Subject: [PATCH 60/84] Change private member functions of QueryPlan to public --- src/Processors/QueryPlan/QueryPlan.h | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 2c1b9cb52e9a..21793fbd7ea5 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -77,6 +77,8 @@ class QueryPlan bool isCompleted() const; /// Tree is not empty and root hasOutputStream() const DataStream & getCurrentDataStream() const; /// Checks that (isInitialized() && !isCompleted()) + void checkInitialized() const; + void checkNotCompleted() const; void optimize(const QueryPlanOptimizationSettings & optimization_settings); void reset(); @@ -141,9 +143,6 @@ class QueryPlan Nodes nodes; Node * root = nullptr; - void checkInitialized() const; - void checkNotCompleted() const; - /// Those fields are passed to QueryPipeline. size_t max_threads = 0; std::vector interpreter_context; From d897c07f8622e3f2bf56a3ae6ffbc838c4697597 Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Sun, 23 Jan 2022 04:35:10 +0000 Subject: [PATCH 61/84] Allow ATTACH/DROP PARTITION; disallow FETCH PARTITION; disallow any ALTER PART --- src/Interpreters/InterpreterAlterQuery.cpp | 7 ++++--- src/Interpreters/executeDDLQueryOnCluster.cpp | 2 -- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 729a495987fe..8cc9e1f408e4 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -33,6 +33,7 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; extern const int NOT_IMPLEMENTED; extern const int TABLE_IS_READ_ONLY; + extern const int UNSUPPORTED_METHOD; } @@ -67,9 +68,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (typeid_cast(database.get()) && !getContext()->getClientInfo().is_replicated_database_internal - && !alter.isAttachAlter() - && !alter.isFetchAlter() - && !alter.isDropPartitionAlter()) + && !alter.isFetchAlter()) { auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name); guard->releaseTableLock(); @@ -95,6 +94,8 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) for (const auto & child : alter.command_list->children) { auto * command_ast = child->as(); + if (command_ast->part) + throw Exception("Manipulating Part is not supported", ErrorCodes::UNSUPPORTED_METHOD); if (auto alter_command = AlterCommand::parse(command_ast)) { alter_commands.emplace_back(std::move(*alter_command)); diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index c3b8cc5c6776..baa685ca4b98 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -37,8 +37,6 @@ bool isSupportedAlterType(int type) { assert(type != ASTAlterCommand::NO_TYPE); static const std::unordered_set unsupported_alter_types{ - /// It's dangerous, because it may duplicate data if executed on multiple replicas. We can allow it after #18978 - ASTAlterCommand::ATTACH_PARTITION, /// Usually followed by ATTACH PARTITION ASTAlterCommand::FETCH_PARTITION, /// Logical error From 76a464a6b330ea6f65ac73af0a442f9b21d8e10a Mon Sep 17 00:00:00 2001 From: Chao Ma Date: Thu, 20 Jan 2022 16:24:59 +0800 Subject: [PATCH 62/84] Fix hang when alter table where with materialized column --- src/Interpreters/MutationsInterpreter.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 471ad67d4e75..a7cd4c20dcc6 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -231,8 +231,8 @@ bool isStorageTouchedByMutations( PullingPipelineExecutor executor(io.pipeline); Block block; - while (!block.rows()) - executor.pull(block); + while (executor.pull(block)); + if (!block.rows()) return false; else if (block.rows() != 1) From 03e0c202b76da9717ece929e541436fc88f21d84 Mon Sep 17 00:00:00 2001 From: Chao Ma Date: Sat, 22 Jan 2022 14:37:02 +0800 Subject: [PATCH 63/84] Fix create as select multi results and no table error --- src/Databases/DatabaseReplicated.cpp | 1 + src/Interpreters/DDLTask.h | 2 ++ src/Interpreters/InterpreterCreateQuery.cpp | 7 ++++++- src/Interpreters/InterpreterCreateQuery.h | 5 +++-- src/Interpreters/executeDDLQueryOnCluster.cpp | 16 ++++++++++++++++ 5 files changed, 28 insertions(+), 3 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 1018a468f374..1ef6d004b2ae 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -458,6 +458,7 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex entry.query = queryToString(query); entry.initiator = ddl_worker->getCommonHostID(); entry.setSettingsIfRequired(query_context); + entry.query_ptr = query->clone(); String node_path = ddl_worker->tryEnqueueAndExecuteEntry(entry, query_context); Strings hosts_to_wait = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index ee49274707a8..3bae37ceb8cb 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -72,6 +72,8 @@ struct DDLLogEntry String initiator; // optional std::optional settings; + ASTPtr query_ptr; + void setSettingsIfRequired(ContextPtr context); String toString() const; void parse(const String & data); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f0dda070ffc5..580fff7ee43b 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1261,12 +1261,17 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, } } -BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create) +BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create, bool need_fill) { /// If the query is a CREATE SELECT, insert the data into the table. if (create.select && !create.attach && !create.is_ordinary_view && !create.is_live_view && (!create.is_materialized_view || create.is_populate)) { + auto database = DatabaseCatalog::instance().getDatabase(create.database); + if (getContext()->getSettingsRef().allow_experimental_database_replicated && database->getEngineName() == "Replicated" + && !need_fill) + return {}; + auto insert = std::make_shared(); insert->table_id = {create.database, create.table, create.uuid}; insert->select = create.select->clone(); diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 89d27a30555d..45fe5df3e444 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -66,6 +66,9 @@ class InterpreterCreateQuery : public IInterpreter, WithMutableContext void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const override; + /// Inserts data in created table if it's CREATE ... SELECT + BlockIO fillTableIfNeeded(const ASTCreateQuery & create, bool need_fill = false); + private: struct TableProperties { @@ -87,8 +90,6 @@ class InterpreterCreateQuery : public IInterpreter, WithMutableContext /// Create IStorage and add it to database. If table already exists and IF NOT EXISTS specified, do nothing and return false. bool doCreateTable(ASTCreateQuery & create, const TableProperties & properties); BlockIO doCreateOrReplaceTable(ASTCreateQuery & create, const InterpreterCreateQuery::TableProperties & properties); - /// Inserts data in created table if it's CREATE ... SELECT - BlockIO fillTableIfNeeded(const ASTCreateQuery & create); void assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const; diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index baa685ca4b98..0595a19d28c0 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -160,6 +162,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, entry.hosts = std::move(hosts); entry.query = queryToString(query_ptr); entry.initiator = ddl_worker.getCommonHostID(); + entry.query_ptr = query_ptr->clone(); entry.setSettingsIfRequired(context); String node_path = ddl_worker.enqueueQuery(entry); @@ -185,6 +188,7 @@ class DDLQueryStatusSource final : public SourceWithProgress std::pair parseHostAndPort(const String & host_id) const; String node_path; + DDLLogEntry entry; ContextPtr context; Stopwatch watch; Poco::Logger * log; @@ -250,6 +254,7 @@ DDLQueryStatusSource::DDLQueryStatusSource( const String & zk_node_path, const DDLLogEntry & entry, ContextPtr context_, const std::optional & hosts_to_wait) : SourceWithProgress(getSampleBlock(context_, hosts_to_wait.has_value()), true) , node_path(zk_node_path) + , entry(entry) , context(context_) , watch(CLOCK_MONOTONIC_COARSE) , log(&Poco::Logger::get("DDLQueryStatusInputStream")) @@ -293,7 +298,18 @@ Chunk DDLQueryStatusSource::generate() assert(num_hosts_finished <= waiting_hosts.size()); if (all_hosts_finished || timeout_exceeded) + { + if (auto * create_query = entry.query_ptr->as()) + { + if (create_query->database.empty()) + create_query->database = context->getCurrentDatabase(); + LOG_DEBUG(log, "Fill table {}.{} when all hosts has finished table creation", create_query->database, create_query->table); + BlockIO fill_io = InterpreterCreateQuery(entry.query_ptr, context->getQueryContext()).fillTableIfNeeded(*create_query, true); + executeTrivialBlockIO(fill_io, context); + } + return {}; + } auto zookeeper = context->getZooKeeper(); size_t try_number = 0; From 60f09f72a13a7a1c98c527f78bccadd0400eed19 Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Mon, 24 Jan 2022 05:54:01 +0000 Subject: [PATCH 64/84] Add distributed SYSTEM DDL --- src/Interpreters/InterpreterSystemQuery.cpp | 33 +++++++++++++++++++-- src/Interpreters/InterpreterSystemQuery.h | 2 ++ 2 files changed, 33 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index e34d974fa805..5de840b12e25 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include #include #include @@ -192,6 +193,7 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, } } } + commitSystemQuery(); } @@ -205,11 +207,26 @@ BlockIO InterpreterSystemQuery::execute() { auto & query = query_ptr->as(); + using Type = ASTSystemQuery::Type; + + auto is_support_system_query_ddl = [](Type & type) + { + return Type::STOP_MERGES == type || Type::START_MERGES == type || Type::STOP_TTL_MERGES == type || Type::START_TTL_MERGES == type + || Type::FLUSH_LOGS == type; + }; + + if (!getContext()->getClientInfo().is_replicated_database_internal && is_support_system_query_ddl(query.type)) + { + auto * ptr = typeid_cast( + DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); + if (!ptr) + throw Exception("The default database is not Replicated engine", ErrorCodes::LOGICAL_ERROR); + return ptr->tryEnqueueReplicatedDDL(query_ptr, getContext()); + } + if (!query.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccessForDDLOnCluster()); - using Type = ASTSystemQuery::Type; - /// Use global context with fresh system profile settings auto system_context = Context::createCopy(getContext()->getGlobalContext()); system_context->setSetting("profile", getContext()->getSystemProfileName()); @@ -905,4 +922,16 @@ void InterpreterSystemQuery::extendQueryLogElemImpl(QueryLogElement & elem, cons elem.query_kind = "System"; } +void InterpreterSystemQuery::commitSystemQuery() const +{ + if (getContext()->getClientInfo().is_replicated_database_internal) + { + auto * ptr = typeid_cast( + DatabaseCatalog::instance().getDatabase(getContext()->getConfigRef().getString("default_database", "default")).get()); + if (!ptr) + throw Exception("The default database is not Replicated engine", ErrorCodes::LOGICAL_ERROR); + ptr->commitDatabase(getContext()); + } +} + } diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 6d1ad78a991c..a534c82cb13e 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -61,6 +61,8 @@ class InterpreterSystemQuery : public IInterpreter, WithMutableContext void startStopAction(StorageActionBlockType action_type, bool start); void extendQueryLogElemImpl(QueryLogElement &, const ASTPtr &, ContextPtr) const override; + + void commitSystemQuery() const; }; From 7c7f46a3dbbf1b2e3dd594027f042983d8871763 Mon Sep 17 00:00:00 2001 From: madianjun Date: Mon, 24 Jan 2022 14:14:16 +0800 Subject: [PATCH 65/84] Schedule intermediate stages on local compute node --- src/Processors/QueryPlan/DistributedPlanner.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/DistributedPlanner.cpp b/src/Processors/QueryPlan/DistributedPlanner.cpp index a99295e30813..8fc744aabe11 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.cpp +++ b/src/Processors/QueryPlan/DistributedPlanner.cpp @@ -404,6 +404,7 @@ bool DistributedPlanner::scheduleStages(PlanResult & plan_result) query_plan.root = parent->root_node; stages.pop_back(); is_result_stage_moved_forward = true; + return; } } LOG_DEBUG(log, "Schedule stage {} to 1 worker(local).", stage->id); @@ -411,9 +412,8 @@ bool DistributedPlanner::scheduleStages(PlanResult & plan_result) } /// Intermediate stage. - stage->workers.reserve(compute_replicas.size()); - stage->workers.insert(stage->workers.end(), compute_replicas.begin(), compute_replicas.end()); - LOG_DEBUG(log, "Schedule stage {} to {} workers.", stage->id, stage->workers.size()); + stage->workers.emplace_back(std::make_shared(my_replica)); + LOG_DEBUG(log, "Schedule stage {} to 1 worker(local).", stage->id); }; struct Frame From 658947802d96fb2ed19a53d1a686ef6add3e6408 Mon Sep 17 00:00:00 2001 From: Chao Ma Date: Mon, 24 Jan 2022 09:37:05 +0800 Subject: [PATCH 66/84] Fix group by const error --- src/Interpreters/Context.h | 1 + src/Processors/QueryPlan/AggregatingStep.h | 2 ++ src/Processors/QueryPlan/DistributedPlanner.cpp | 13 +++++++++++++ src/Processors/QueryPlan/DistributedPlanner.h | 1 + src/Server/GRPCServer.cpp | 3 ++- src/Server/grpc_protos/clickhouse_grpc.proto | 1 + 6 files changed, 20 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 48d3e6ddd7c8..b2cf062808a5 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -305,6 +305,7 @@ class Context: public std::enable_shared_from_this String node_id; /// This is myself replica name and grpc port. std::unordered_map>> parent_sources; /// Mapping of parent id and its workers. std::vector> sinks; /// Point to workers that receiving data. + bool empty_result_for_aggregation_by_empty_set; }; // Top-level OpenTelemetry trace context for the query. Makes sense only for a query context. diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index c16d916f1ed7..24436f6b2c0a 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -42,6 +42,8 @@ class AggregatingStep : public ITransformingStep void describePipeline(FormatSettings & settings) const override; const Aggregator::Params & getParams() const { return params; } + void setEmptyResultForAggregationByEmptySet(bool empty_result_for_aggregation_by_empty_set_) { params.empty_result_for_aggregation_by_empty_set = + empty_result_for_aggregation_by_empty_set_;} private: Aggregator::Params params; diff --git a/src/Processors/QueryPlan/DistributedPlanner.cpp b/src/Processors/QueryPlan/DistributedPlanner.cpp index 8fc744aabe11..d7ab4ca8a0da 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.cpp +++ b/src/Processors/QueryPlan/DistributedPlanner.cpp @@ -146,6 +146,11 @@ void DistributedPlanner::buildStages() { ++stage_id; last_stage = createStage(stage_id, parent_stages, last_node, leaf_nodes); + if (result.child_aggregating_step) + { + last_stage->empty_result_for_aggregation_by_empty_set + = result.child_aggregating_step->getParams().empty_result_for_aggregation_by_empty_set; + } /// The new stage is parent of current node's stage. parent_stages.push(last_stage); @@ -541,6 +546,7 @@ bool DistributedPlanner::scheduleStages(PlanResult & plan_result) /// Fill with data related to each stage. query_info.set_query_id(context->generateQueryId()); query_info.set_stage_id(stage.id); + query_info.set_empty_result_for_aggregation_by_empty_set(stage.empty_result_for_aggregation_by_empty_set); /// TODO: Not all stages need external tables, so choose the ones that are necessary, at least for leaf stages. /// Fill external tables(reference from Connection.cpp: void Connection::sendExternalTablesData(ExternalTablesData & data)): @@ -899,6 +905,13 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) if (!result.child_aggregating_step->getParams().optimize_trivial_count) { auto aggregating_step = std::make_unique(*result.child_aggregating_step); + if (query_distributed_plan_info.empty_result_for_aggregation_by_empty_set) + { + LOG_DEBUG(log, "Set (aggregating_step) param (empty_result_for_aggregation_by_empty_set) to {} for distributed query plan to keep compute and store nodes same", + query_distributed_plan_info.empty_result_for_aggregation_by_empty_set); + aggregating_step->setEmptyResultForAggregationByEmptySet( + query_distributed_plan_info.empty_result_for_aggregation_by_empty_set); + } replaceStep(std::move(aggregating_step), last_node); } /// If optimize trivial count, remove AggregatingStep. diff --git a/src/Processors/QueryPlan/DistributedPlanner.h b/src/Processors/QueryPlan/DistributedPlanner.h index ebf6aa92dc47..f6ca21fc1201 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.h +++ b/src/Processors/QueryPlan/DistributedPlanner.h @@ -24,6 +24,7 @@ class DistributedPlanner { bool is_leaf_stage = false; /// Current stage is a leaf stage if it has any leaf node reading data from storage(not from remote). bool maybe_has_view_source = false; /// Current stage reads data to trigger materialized view. bool has_input_function = false; + bool empty_result_for_aggregation_by_empty_set = false; }; /// Note: do not use vector, otherwise pointers to elements in it will be invalidated when vector increases. diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 5ba9aaf13606..98c9be372657 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1172,7 +1172,8 @@ namespace .stage_id = query_info->stage_id(), .node_id = query_info->node_id(), .parent_sources = std::move(parent_sources), - .sinks = sinks }; + .sinks = sinks, + .empty_result_for_aggregation_by_empty_set = query_info->empty_result_for_aggregation_by_empty_set()}; query_context->setQueryPlanFragmentInfo(std::move(fragment_info)); } diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index bb586779c346..eac3ad7567a3 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -125,6 +125,7 @@ message QueryInfo { repeated string sinks = 24; bool has_view_source = 25; bool has_input_function = 26; + bool empty_result_for_aggregation_by_empty_set = 27; } enum LogsLevel { From 556228ef831c4d8859c15ba072b004c6f27ac211 Mon Sep 17 00:00:00 2001 From: madianjun Date: Mon, 24 Jan 2022 10:46:33 +0800 Subject: [PATCH 67/84] Build same original query plan when trivial count is optimized, and skip executeWhere --- src/Interpreters/InterpreterSelectQuery.cpp | 17 +++++++---------- src/Processors/QueryPlan/DistributedPlanner.cpp | 10 +++++++--- src/Processors/QueryPlan/DistributedPlanner.h | 1 + 3 files changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a3371d0726a9..a8a8d52ac7ab 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1204,7 +1204,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

getRunningMode() == Context::RunningMode::STORE - && syntax_analyzer_result->optimize_trivial_count + syntax_analyzer_result->optimize_trivial_count && (settings.max_parallel_replicas <= 1) && storage && storage->getName() != "MaterializedMySQL" @@ -1850,13 +1849,11 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc auto source = std::make_shared(block_with_count); auto prepared_count = std::make_unique(Pipe(std::move(source)), context); prepared_count->setStepDescription("Optimized trivial count"); - query_plan.addStep(std::move(prepared_count)); - if (context->isInitialQuery()) - { - /// If initial query is running on store worker, skip first stage. - from_stage = QueryProcessingStage::WithMergeableState; - analysis_result.first_stage = false; - } + const auto & ast = query_info.query->as(); + InterpreterParamsPtr interpreter_params = std::make_shared(context, ast); + query_plan.addStep(std::move(prepared_count), std::move(interpreter_params)); + /// Build query plan for the first stage both on compute and store nodes, therefore we can get the same original query plan. + /// If trivial count is optimized, skip executeWhere. analysis_result.optimize_trivial_count = true; return; } diff --git a/src/Processors/QueryPlan/DistributedPlanner.cpp b/src/Processors/QueryPlan/DistributedPlanner.cpp index d7ab4ca8a0da..cee4d6ec821b 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.cpp +++ b/src/Processors/QueryPlan/DistributedPlanner.cpp @@ -65,6 +65,7 @@ void DistributedPlanner::checkShuffle(QueryPlan::Node * current_node, QueryPlan: { LOG_DEBUG(log, "Check shuffle: child node is AggregatingStep"); result.is_shuffle = true; + result.grandchild_step = child_node->children.front()->step.get(); return; } @@ -826,8 +827,11 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) aggregating_step = std::make_unique(*result.child_aggregating_step); } /// The aggregating_step header will include aggregate function. - const auto & header = result.child_aggregating_step == nullptr ? last_node->step->getOutputStream().header - : aggregating_step->getOutputStream().header; + const auto & header = result.child_aggregating_step == nullptr + ? last_node->step->getOutputStream().header + : ((result.child_aggregating_step->getParams().optimize_trivial_count && result.grandchild_step) + ? result.grandchild_step->getOutputStream().header + : aggregating_step->getOutputStream().header); /// Create DistributedSourceStep. assert(header); @@ -914,7 +918,7 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) } replaceStep(std::move(aggregating_step), last_node); } - /// If optimize trivial count, remove AggregatingStep. + /// If optimize trivial count, remove AggregatingStep. else { LOG_DEBUG(log, "Remove step: {}, stage: {}", result.child_aggregating_step->getName(), stage_id); diff --git a/src/Processors/QueryPlan/DistributedPlanner.h b/src/Processors/QueryPlan/DistributedPlanner.h index f6ca21fc1201..c59aa8a641fa 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.h +++ b/src/Processors/QueryPlan/DistributedPlanner.h @@ -53,6 +53,7 @@ class DistributedPlanner { SortingStep * child_sorting_step = nullptr; LimitStep * current_limit_step = nullptr; LimitStep * child_limit_step = nullptr; + IQueryPlanStep * grandchild_step = nullptr; }; void checkShuffle(QueryPlan::Node * current_node, QueryPlan::Node * child_node, CheckShuffleResult & result); From 75f22e6806cacf795ad78bacbfcb449e49a340a3 Mon Sep 17 00:00:00 2001 From: madianjun Date: Mon, 24 Jan 2022 20:09:15 +0800 Subject: [PATCH 68/84] Clean grpc exception periodically thrown by producer; let consumer catch the root cause exception --- src/Server/GRPCServer.cpp | 135 +++++++++++++++++++++++++++++++++----- 1 file changed, 118 insertions(+), 17 deletions(-) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 98c9be372657..124f79069680 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -46,6 +46,7 @@ #include +using namespace std::chrono_literals; using GRPCService = clickhouse::grpc::ClickHouse::AsyncService; using GRPCQueryInfo = clickhouse::grpc::QueryInfo; using GRPCTicket = clickhouse::grpc::Ticket; @@ -724,6 +725,8 @@ namespace class InnerMap { public: + InnerMap(String name): log(&Poco::Logger::get(name)) {} + using Impl = std::unordered_map; auto get(const String & key_) { @@ -734,7 +737,7 @@ namespace return std::pair{it->second, true}; } - auto insert(const String& key_, Value && value_) + auto insert(const String & key_, Value && value_) { std::lock_guard lock(mutex); return container.emplace(key_, value_); @@ -746,10 +749,26 @@ namespace return container.erase(key_); } + using Callback = bool(*)(const Value & it, const std::chrono::time_point & now); + void eraseAll(Callback callback, const std::chrono::time_point & now) + { + std::lock_guard lock(mutex); + for (auto it = container.begin(); it != container.end();) + { + if (callback(it->second, now)) + { + LOG_DEBUG(log, "Clean entry for key {}", it->first); + container.erase(it++); + } + else + ++it; + } + } private: Impl container; std::mutex mutex; + Poco::Logger * log = nullptr; }; class QueryInfoWrapper @@ -808,7 +827,6 @@ namespace std::atomic finish{false}; std::atomic cancel{false}; std::chrono::seconds wait_timeout_seconds{600}; - Exception exception; }; void QueryInfoWrapper::notifyHeader(Block header_) @@ -888,13 +906,29 @@ namespace cv_producer.notify_one(); } - using QueryInfoMap = InnerMap>; + using QueryInfoWrapperMap = InnerMap>; + + class ExceptionWrapper + { + public: + ExceptionWrapper(const Exception & exception_, const std::chrono::time_point & timestamp_) + : exception(exception_), timestamp(timestamp_) {} + Exception exception; + std::chrono::time_point timestamp; + }; + using ExceptionWrapperMap = InnerMap>; /// Handles a connection after a responder is started (i.e. after getting a new call). class Call { public: - Call(CallType call_type_, std::unique_ptr responder_, IServer & iserver_, Poco::Logger * log_); + Call( + CallType call_type_, + std::unique_ptr responder_, + IServer & iserver_, + std::shared_ptr & query_info_map, + std::shared_ptr & exception_map, + Poco::Logger * log_); ~Call(); void start(const std::function & on_finish_call_callback); @@ -944,6 +978,10 @@ namespace const CallType call_type; std::unique_ptr responder; IServer & iserver; + std::shared_ptr & query_info_map; + String query_info_key; + std::shared_ptr query_info_wrapper; + std::shared_ptr & exception_map; Poco::Logger * log = nullptr; std::optional session; @@ -967,10 +1005,6 @@ namespace GRPCResult result; GRPCTicket ticket; - static std::unique_ptr query_info_map; - String query_info_key; - std::shared_ptr query_info_wrapper; - bool initial_query_info_read = false; bool initial_ticket_read = false; bool finalize = false; @@ -1005,10 +1039,19 @@ namespace ThreadFromGlobalPool call_thread; }; - std::unique_ptr Call::query_info_map = std::make_unique(); - - Call::Call(CallType call_type_, std::unique_ptr responder_, IServer & iserver_, Poco::Logger * log_) - : call_type(call_type_), responder(std::move(responder_)), iserver(iserver_), log(log_) + Call::Call( + CallType call_type_, + std::unique_ptr responder_, + IServer & iserver_, + std::shared_ptr & query_info_map_, + std::shared_ptr & exception_map_, + Poco::Logger * log_) + : call_type(call_type_) + , responder(std::move(responder_)) + , iserver(iserver_) + , query_info_map(query_info_map_) + , exception_map(exception_map_) + , log(log_) { } @@ -1339,6 +1382,13 @@ namespace } else { + const auto & check_exception_result = exception_map->get(query_info_key); + if (check_exception_result.second) + { + LOG_WARNING(log, "Query info key {} not found, and catch exception from producer.", query_info_key); + throw Exception(check_exception_result.first->exception); + } + if (is_cancel) /// Plan fragment maybe done. LOG_INFO(log, "Query info key {} to be cancelled does not exist, so ignore it.", query_info_key); else @@ -1781,8 +1831,9 @@ namespace } /// Throw exception which is from producer. - if (query_info_wrapper->exception.code()) - throw Exception(query_info_wrapper->exception); + const auto & check_exception_result = exception_map->get(query_info_key); + if (check_exception_result.second) + throw Exception(check_exception_result.first->exception); if (!query_info_wrapper->cancel) { @@ -1863,7 +1914,10 @@ namespace if (query_info_wrapper) { LOG_DEBUG(log, "{} producer has an exception", query_info_key); - query_info_wrapper->exception = exception; + auto const & now = std::chrono::steady_clock::now(); + auto exception_wrapper = std::make_shared(exception, now); + exception_map->insert(query_info_key, std::move(exception_wrapper)); + cancelPlanFragment(); query_info_wrapper->notifyFinish(); } @@ -2265,12 +2319,19 @@ namespace class GRPCServer::Runner { public: - explicit Runner(GRPCServer & owner_) : owner(owner_) {} + explicit Runner(GRPCServer & owner_) : owner(owner_) + { + query_info_map = std::make_shared("QueryInfoWrapperMap"); + exception_map = std::make_shared("ExceptionWrapperMap"); + } ~Runner() { if (queue_thread.joinable()) queue_thread.join(); + + if (clean_thread.joinable()) + clean_thread.join(); } void start() @@ -2290,6 +2351,19 @@ class GRPCServer::Runner } }; queue_thread = ThreadFromGlobalPool{runner_function}; + + auto cleanner_function = [this] + { + try + { + clean(); + } + catch (...) + { + tryLogCurrentException("GRPCServer"); + } + }; + clean_thread = ThreadFromGlobalPool{cleanner_function}; } void stop() { stopReceivingNewCalls(); } @@ -2336,7 +2410,7 @@ class GRPCServer::Runner { /// Connection established and the responder has been started. /// So we pass this responder to a Call and make another responder for next connection. - auto new_call = std::make_unique(call_type, std::move(responder), owner.iserver, owner.log); + auto new_call = std::make_unique(call_type, std::move(responder), owner.iserver, query_info_map, exception_map, owner.log); auto * new_call_ptr = new_call.get(); current_calls[new_call_ptr] = std::move(new_call); new_call_ptr->start([this, new_call_ptr]() { onFinishCall(new_call_ptr); }); @@ -2387,6 +2461,29 @@ class GRPCServer::Runner } } + static bool shouldClean( + const std::shared_ptr & execption_wrapper, + const std::chrono::time_point & now) + { + auto elapsed = std::chrono::duration_cast(now - execption_wrapper->timestamp); + if (elapsed > std::chrono::seconds(10s)) + return true; + return false; + } + + void clean() + { + using namespace std::chrono_literals; + setThreadName("GRPCServerCleaner"); + while (!should_stop) + { + auto const & now = std::chrono::steady_clock::now(); + exception_map->eraseAll(shouldClean, now); + + std::this_thread::sleep_for(10s); + } + } + GRPCServer & owner; ThreadFromGlobalPool queue_thread; std::vector> responders_for_new_calls; @@ -2394,6 +2491,10 @@ class GRPCServer::Runner std::vector> finished_calls; bool should_stop = false; mutable std::mutex mutex; + + std::shared_ptr query_info_map; + std::shared_ptr exception_map; + ThreadFromGlobalPool clean_thread; }; From e2e5697ac4a29e2eaae96fcea114daf4c64f70b5 Mon Sep 17 00:00:00 2001 From: madianjun Date: Tue, 25 Jan 2022 19:05:00 +0800 Subject: [PATCH 69/84] Support DISTINCT --- .../QueryPlan/DistributedPlanner.cpp | 101 +++++++++++++----- src/Processors/QueryPlan/DistributedPlanner.h | 31 +++++- src/Processors/QueryPlan/QueryPlan.h | 6 -- 3 files changed, 104 insertions(+), 34 deletions(-) diff --git a/src/Processors/QueryPlan/DistributedPlanner.cpp b/src/Processors/QueryPlan/DistributedPlanner.cpp index cee4d6ec821b..2cd9fd78d418 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.cpp +++ b/src/Processors/QueryPlan/DistributedPlanner.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -34,18 +35,32 @@ DistributedPlanner::DistributedPlanner(QueryPlan & query_plan_, const ContextMut { } -void DistributedPlanner::checkShuffle(QueryPlan::Node * current_node, QueryPlan::Node * child_node, CheckShuffleResult & result) +void DistributedPlanner::checkShuffle( + QueryPlan::Node * current_node, + QueryPlan::Node * child_node, + CheckShuffleResult & result, + StageSeq & stage_seq) { - /// Cases: - /// 1. child node is aggregate step. - /// 2. current node is limit step, child node is sort step: no need shuffle. - /// 3. current node is not limit step, child node is sort step: need shuffle. - /// 4. child node is limit step: need shuffle. + /// Cases requiring special attention: + /// 1. Distinct: + /// distinct => [(shuffle)] => distinct + /// distinct => [(shuffle)] => distinct => limit + /// + /// 2. Limit(limit should be pushdown): + /// => limit => [(shuffle) => limit] + /// sort => limit => [(shuffle) => sort => limit] + /// + /// 3. Sort: + /// distinct => sort => [(shuffle) => sort] => distinct + /// distinct => sort => [(shuffle) => sort] => distinct => limit + /// + /// Note: The content in "[]" is added to the edge between two stages. result.current_union_step = dynamic_cast(current_node->step.get()); if (result.current_union_step) { LOG_DEBUG(log, "Check shuffle: child node is UnionStep"); result.is_shuffle = true; + stage_seq = StageSeq::STAGE1; return; } @@ -57,33 +72,62 @@ void DistributedPlanner::checkShuffle(QueryPlan::Node * current_node, QueryPlan: /// Only broadcast right side. if (child_node == current_node->children[1]) result.is_shuffle = true; + stage_seq = StageSeq::STAGE1; return; } result.child_aggregating_step = dynamic_cast(child_node->step.get()); if (result.child_aggregating_step) { + /// From : AggregatingStep => + /// To : AggregatingStep(partial) [=> MergingAggregatedStep(final)] => LOG_DEBUG(log, "Check shuffle: child node is AggregatingStep"); - result.is_shuffle = true; result.grandchild_step = child_node->children.front()->step.get(); + result.is_shuffle = true; + stage_seq = StageSeq::STAGE2; return; } result.child_sorting_step = dynamic_cast(child_node->step.get()); if (result.child_sorting_step) { + /// From : SortingStep => Not (LimitStep) => + /// To : SortingStep(partial) [=> SortingStep(final)] => Not (LimitStep) => LOG_DEBUG(log, "Check shuffle: child node is SortingStep"); result.current_limit_step = dynamic_cast(current_node->step.get()); + if (!result.current_limit_step) + { + result.is_shuffle = true; + stage_seq = StageSeq::STAGE2; + } + return; } - else + + if ((result.child_distinct_step = dynamic_cast(child_node->step.get()))) { - result.child_limit_step = dynamic_cast(child_node->step.get()); - if (result.child_limit_step) - LOG_DEBUG(log, "Check shuffle: child node is LimitStep"); + LOG_DEBUG(log, "Check shuffle: child node is DistinctStep"); + result.current_distinct_step = dynamic_cast(current_node->step.get()); + if (result.current_distinct_step) + { + /// DistinctStep(partial) => (shuffle) => DistinctStep(final) => + result.is_shuffle = true; + stage_seq = StageSeq::STAGE2; + } + return; } - if ((result.child_sorting_step && !result.current_limit_step) || result.child_limit_step) + if ((result.child_limit_step = dynamic_cast(child_node->step.get()))) + { + LOG_DEBUG(log, "Check shuffle: child node is LimitStep"); + assert(child_node->children.size() == 1); + result.grandchild_sorting_step = dynamic_cast(child_node->children[0]->step.get()); + /// If grandchild is SortingStep: + /// From : SortingStep => LimitStep => + /// To : SortingStep(partial) => LimitStep(partial) [=> SortingStep(final) => LimitStep(final)] => result.is_shuffle = true; + stage_seq = StageSeq::STAGE2; + return; + } } void DistributedPlanner::buildStages() @@ -133,6 +177,7 @@ void DistributedPlanner::buildStages() Stage * last_stage = nullptr; std::stack parent_stages; std::stack leaf_nodes; + StageSeq stage_seq = StageSeq::STAGE1; while (!stack.empty()) { @@ -142,7 +187,7 @@ void DistributedPlanner::buildStages() { /// This is shuffle, create a new stage for child_node. CheckShuffleResult result; - checkShuffle(frame.node, last_node, result); + checkShuffle(frame.node, last_node, result, stage_seq); if (result.is_shuffle) { ++stage_id; @@ -201,7 +246,7 @@ void DistributedPlanner::buildStages() if (next_child == frame.node->children.size() || (dynamic_cast(frame.node->step.get()) && frame.visited_children == 1)) { - LOG_DEBUG(log, "Visit step: {}", frame.node->step->getName()); + LOG_DEBUG(log, "[{}]Visit step: {}", getStageSeqName(stage_seq), frame.node->step->getName()); last_node = frame.node; one_child_is_visited = true; stack.pop(); @@ -770,6 +815,7 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) /// Used for locating the plan fragment. int stage_id = -1; QueryPlan::Node * last_node = nullptr; + StageSeq stage_seq = StageSeq::STAGE1; while (!stack.empty()) { @@ -790,7 +836,7 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) } CheckShuffleResult result; - checkShuffle(frame.node, last_node, result); + checkShuffle(frame.node, last_node, result, stage_seq); /// This is a shuffle dependency between current node and the last visited child. if (result.is_shuffle) @@ -867,23 +913,24 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) addStep(std::move(merging_aggregated), "Merge aggregated streams for distributed AGGREGATE", new_node); } - /// If parent stage has order by, add SortingStep. - if (result.child_sorting_step) + else if (result.child_sorting_step) { auto merging_sorted = std::make_unique(new_node->step->getOutputStream(), *result.child_sorting_step); addStep(std::move(merging_sorted), "Merge sorted streams for distributed ORDER BY", new_node); } - + /// If parent stage has distinct, do nothing. + else if (result.child_distinct_step) + { + /// Do nothing + } /// If parent stage has limit, add LimitStep. - if (result.child_limit_step) + else if (result.child_limit_step) { - assert(last_node->children.size() == 1); - const SortingStep * grandchild_sorting_step = dynamic_cast(last_node->children[0]->step.get()); - if (grandchild_sorting_step) + if (result.grandchild_sorting_step) { auto merging_sorted - = std::make_unique(new_node->step->getOutputStream(), *grandchild_sorting_step); + = std::make_unique(new_node->step->getOutputStream(), *result.grandchild_sorting_step); addStep(std::move(merging_sorted), "Merge sorted streams for distributed ORDER BY", new_node); } @@ -911,7 +958,7 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) auto aggregating_step = std::make_unique(*result.child_aggregating_step); if (query_distributed_plan_info.empty_result_for_aggregation_by_empty_set) { - LOG_DEBUG(log, "Set (aggregating_step) param (empty_result_for_aggregation_by_empty_set) to {} for distributed query plan to keep compute and store nodes same", + LOG_DEBUG(log, "Set empty_result_for_aggregation_by_empty_set to true for AggregatingStep", query_distributed_plan_info.empty_result_for_aggregation_by_empty_set); aggregating_step->setEmptyResultForAggregationByEmptySet( query_distributed_plan_info.empty_result_for_aggregation_by_empty_set); @@ -925,7 +972,7 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) last_node = last_node->children[0]; } } - /// If limit step is pushed down, collect (limit + offset) rows. + /// If limit step is pushed down, collect (limit + offset) rows. else if (result.child_limit_step) result.child_limit_step->resetLimitAndOffset(); @@ -943,7 +990,7 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) if (next_child == frame.node->children.size() || (dynamic_cast(frame.node->step.get()) && frame.visited_children == 1)) { - LOG_DEBUG(log, "Visit step: {}", frame.node->step->getName()); + LOG_DEBUG(log, "[{}]Visit step: {}", getStageSeqName(stage_seq), frame.node->step->getName()); last_node = frame.node; one_child_is_visited = true; stack.pop(); @@ -1041,7 +1088,7 @@ bool DistributedPlanner::buildDistributedPlan() plan_result.node_id = query_distributed_plan_info.node_id; } - LOG_DEBUG(log, "Result plan fragment:\n{}", debugLocalPlanFragment(plan_result)); + LOG_DEBUG(log, "Local plan fragment:\n{}", debugLocalPlanFragment(plan_result)); return true; } diff --git a/src/Processors/QueryPlan/DistributedPlanner.h b/src/Processors/QueryPlan/DistributedPlanner.h index c59aa8a641fa..3a4204048ada 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.h +++ b/src/Processors/QueryPlan/DistributedPlanner.h @@ -5,6 +5,14 @@ namespace DB { +class UnionStep; +class JoinStep; +class AggregatingStep; +class SortingStep; +class LimitStep; +class DistinctStep; +class DistributedSourceStep; + class DistributedPlanner { public: DistributedPlanner(QueryPlan & query_plan_, const ContextMutablePtr & context); @@ -12,6 +20,24 @@ class DistributedPlanner { bool buildDistributedPlan(); private: + enum class StageSeq + { + STAGE0, /// Read data source + STAGE1, /// Partial processing + STAGE2 /// Final processing + }; + + String getStageSeqName(const StageSeq & stage_seq) + { + switch (stage_seq) + { + case StageSeq::STAGE0: return "STAGE0"; + case StageSeq::STAGE1: return "STAGE1"; + case StageSeq::STAGE2: return "STAGE2"; + } + return "Unknown"; + } + struct Stage { int id; /// Current stage id. @@ -51,11 +77,14 @@ class DistributedPlanner { JoinStep * current_join_step = nullptr; AggregatingStep * child_aggregating_step = nullptr; SortingStep * child_sorting_step = nullptr; + SortingStep * grandchild_sorting_step = nullptr; + DistinctStep * current_distinct_step = nullptr; + DistinctStep * child_distinct_step = nullptr; LimitStep * current_limit_step = nullptr; LimitStep * child_limit_step = nullptr; IQueryPlanStep * grandchild_step = nullptr; }; - void checkShuffle(QueryPlan::Node * current_node, QueryPlan::Node * child_node, CheckShuffleResult & result); + void checkShuffle(QueryPlan::Node * current_node, QueryPlan::Node * child_node, CheckShuffleResult & result, StageSeq & stage_seq); struct PlanResult { diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 21793fbd7ea5..b141a763bbff 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -28,12 +28,6 @@ using QueryPlanPtr = std::unique_ptr; class Pipe; -class UnionStep; -class JoinStep; -class AggregatingStep; -class SortingStep; -class LimitStep; - struct QueryPlanOptimizationSettings; struct BuildQueryPipelineSettings; From 9fe470fed53e3abdf19cfac7b843adadb18fadae Mon Sep 17 00:00:00 2001 From: caspian Date: Tue, 25 Jan 2022 12:10:41 +0000 Subject: [PATCH 70/84] Add distributed read/write lock to control the access to database and table --- src/Common/ZooKeeper/DistributedRWLock.cpp | 230 ++++++++++++++++++ src/Common/ZooKeeper/DistributedRWLock.h | 69 ++++++ src/Interpreters/Context.h | 5 +- src/Interpreters/InterpreterAlterQuery.cpp | 8 +- src/Interpreters/InterpreterAlterQuery.h | 4 +- src/Interpreters/InterpreterCreateQuery.cpp | 21 +- src/Interpreters/InterpreterDropQuery.cpp | 14 +- src/Interpreters/InterpreterRenameQuery.cpp | 31 ++- src/Interpreters/InterpreterRenameQuery.h | 4 +- .../StorageMaterializedPostgreSQL.h | 2 +- 10 files changed, 369 insertions(+), 19 deletions(-) create mode 100644 src/Common/ZooKeeper/DistributedRWLock.cpp create mode 100644 src/Common/ZooKeeper/DistributedRWLock.h diff --git a/src/Common/ZooKeeper/DistributedRWLock.cpp b/src/Common/ZooKeeper/DistributedRWLock.cpp new file mode 100644 index 000000000000..12d80074a1a2 --- /dev/null +++ b/src/Common/ZooKeeper/DistributedRWLock.cpp @@ -0,0 +1,230 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace fs = std::filesystem; + +namespace zkutil +{ +std::unordered_set DistributedRWLock::rwlock_node_set; +std::shared_mutex DistributedRWLock::rwlock_node_set_mutex; + +DistributedRWLock::DistributedRWLock(zkutil::ZooKeeperPtr zookeeper_, bool readonly_, const String & path_, std::chrono::seconds timeout) + : zk_cli(zookeeper_) + , readonly(readonly_) + , parent_path(path_) + , wait_timeout_seconds(timeout) + , log(&Poco::Logger::get("DistributedRWLock")) +{ + parent_path = fs::path{parent_path}.lexically_normal(); + + if (parent_path.ends_with("/")) + parent_path.resize(parent_path.size() - 1); + + try_lock_result = LockResult::INIT; +} + +DistributedRWLock::~DistributedRWLock() +{ + { + std::unique_lock lock(rwlock_node_set_mutex); + rwlock_node_set.erase(rwlock_node); + } + + if (!rwlock_node.empty()) + { + zk_cli->remove(rwlock_node); + LOG_TRACE(log, "{} unlock.", rwlock_node); + } +} + +void DistributedRWLock::createRWLockNode() +{ + if (!zk_cli->exists(parent_path)) + { + try + { + zk_cli->createAncestors(fs::path(parent_path) / ""); + } + catch (Coordination::Exception & e) + { + if (e.code != Coordination::Error::ZNODEEXISTS) + { + LOG_WARNING(log, "Create {} got exception: {}.", parent_path, e.message()); + throw e; + } + } + } + + String rwlock_prefix; + if (readonly) + rwlock_prefix = fs::path(parent_path) / ReadLockPrefix; + else + rwlock_prefix = fs::path(parent_path) / WriteLockPrefix; + + rwlock_node = zk_cli->create(rwlock_prefix, "", 3 /* ephemeral and sequential */); + + LOG_TRACE(log, "{} create node success.", rwlock_node); +} + +DistributedRWLockPtr DistributedRWLock::tryLock(zkutil::ZooKeeperPtr zookeeper_, bool readonly_, const String & database, + const String & table, std::chrono::seconds timeout) +{ + auto parent_path = fs::path(LockPathPrefix) / database; + if (!table.empty()) + parent_path /= (TablesDir + table); + auto lock = std::make_shared(zookeeper_, readonly_, parent_path, timeout); + lock->run(); + return lock; +} + +void DistributedRWLock::run() +{ + auto start = std::chrono::system_clock::now(); + createRWLockNode(); + { + std::unique_lock lock(rwlock_node_set_mutex); + rwlock_node_set.insert(rwlock_node); + } + + tryLockImpl(); + while (true) + { + std::unique_lock lock(watch_cond_mutex); + if (watch_cond.wait_until(lock, start + wait_timeout_seconds, [&]() { return try_lock_result > LockResult::INIT; })) + { + switch (try_lock_result) + { + case LockResult::INIT: + continue; + case LockResult::SUCCESS: { + LOG_TRACE(log, "{} try lock success.", rwlock_node); + return; + } + case LockResult::RETRY: { + if (std::chrono::system_clock::now() > (start + wait_timeout_seconds)) + throw Coordination::Exception("Try lock timeout for " + rwlock_node, Coordination::Error::ZOPERATIONTIMEOUT); + try_lock_result = LockResult::INIT; /// Reset the flag + tryLockImpl(); + break; + } + case LockResult::FAILED: + throw Coordination::Exception( + "Try lock failed for " + rwlock_node + ", maybe lost connection with zookeeper", Coordination::Error::ZCONNECTIONLOSS); + } + } + else + throw Coordination::Exception("Try lock timeout for " + rwlock_node, Coordination::Error::ZOPERATIONTIMEOUT); + } +} + +void DistributedRWLock::tryLockImpl() +{ + auto children = zk_cli->getChildren(parent_path); + + /// Only remains the lock items + std::erase_if(children, [](std::string x) { return (!x.starts_with(WriteLockPrefix) && !x.starts_with(ReadLockPrefix)); }); + + auto compare = [](const String & a, const String & b) + { + /// WriteLockPrefix and ReadLockPrefix have the same length, so any one is ok + return std::stoi(a.substr(WriteLockPrefix.size())) < std::stoi(b.substr(WriteLockPrefix.size())); + }; + std::sort(children.begin(), children.end(), compare); + + String watch_node; + if (!readonly) + { + for (size_t i = 0; i < children.size(); i++) + { + if (rwlock_node == (fs::path(parent_path) / children[i])) + { + if (i == 0) + { + try_lock_result = LockResult::SUCCESS; + return; + } + else + { + watch_node = children[i - 1]; + break; + } + } + } + } + else + { + /// Used to locate the write node with the next lowest sequence number. + int write_idx = -1; + for (size_t i = 0; i < children.size(); i++) + { + auto child = children[i]; + if (rwlock_node == (fs::path(parent_path) / child)) + { + if (write_idx == -1) + { + try_lock_result = LockResult::SUCCESS; + return; + } + else + { + watch_node = children[write_idx]; + } + } + + if (child.starts_with(WriteLockPrefix)) + { + write_idx = i; + } + } + } + + if (watch_node.empty()) + throw Coordination::Exception( + "Node " + rwlock_node + " not found in parent path: " + parent_path, Coordination::Error::ZINVALIDSTATE); + + LOG_TRACE(log, "{} will watch {}", rwlock_node, watch_node); + + { + /// The callback must be light weighted. + /// The 'key' is used to to avoid NULL pointer usage of watch_cond_mutex after DistributedRWLock has been destroyed + /// And the 'key' variable itself must be value capture. + auto callback = [&, key = rwlock_node](const Coordination::WatchResponse & resp) + { + /// The lock rwlock_node_set_mutex should be hold until finished using on DistributedRWLock's member. + std::shared_lock lock(rwlock_node_set_mutex); + if (!rwlock_node_set.contains(key)) + { /// Maybe the DistributedRWLock object has been destroyed, so just return. + return; + } + LOG_TRACE(log, "{} watch {} got callback event: type: {}, path: {}, state: {}.", rwlock_node, watch_node, resp.type, resp.path, resp.state); + std::unique_lock result_lock(watch_cond_mutex); + try_lock_result = LockResult::RETRY; + watch_cond.notify_all(); + }; + + try + { + /// Add watch on the node + /// Return true if the node exist and add watch success, false if the node does not exist + /// The callback function will be called when the node was removed + auto watch_success = zk_cli->existsWatch(fs::path(parent_path) / watch_node, nullptr, callback); + if (!watch_success) + { + /// The node does not exists, but maybe there are some other nodes that still hold the lock, so we should retry + try_lock_result = LockResult::RETRY; + return; + } + } + catch (Coordination::Exception & e) + { + try_lock_result = LockResult::FAILED; + throw e; + } + } +} +} diff --git a/src/Common/ZooKeeper/DistributedRWLock.h b/src/Common/ZooKeeper/DistributedRWLock.h new file mode 100644 index 000000000000..8b795500225e --- /dev/null +++ b/src/Common/ZooKeeper/DistributedRWLock.h @@ -0,0 +1,69 @@ +#pragma once +#include +#include + +namespace zkutil +{ +const String ReadLockPrefix = "r-"; +const String WriteLockPrefix = "w-"; +const String LockPathPrefix = "/clickhouse/rwlocks/"; +const String TablesDir = "tables/"; + +class DistributedRWLock; +using DistributedRWLockPtr = std::shared_ptr; + +/// Acquire a distributed read lock or write lock through zookeeper until timeout. +/// Client should use static function tryLock() to acquire a lock. +/// The lock will be released in destructor automically. +/// +/// Implement details: +/// 1. Create an ephemeral and sequential node as the number of the "try lock queue". +/// 2. Get all sequential nodes in the "queue", check if trying lock can be successful, if success returns. +/// 3. Otherwise watch a node (the write node for read lock, the previous read node for write lock) and wait for notification, go to step 2 when got notified. +/// Details can be found here: https://zookeeper.apache.org/doc/current/recipes.html#Shared+Locks +class DistributedRWLock +{ +public: + DistributedRWLock(zkutil::ZooKeeperPtr zookeeper_, bool readonly_, const String & path_, std::chrono::seconds timeout); + ~DistributedRWLock(); + + /// Try to get a lock (readonly_ true for read lock, false for write lock) on a specified zookeeper path. + /// Unlock automically when the returned DistributedRWLockPtr is destructed. + static DistributedRWLockPtr tryLock(zkutil::ZooKeeperPtr zookeeper_, bool readonly_, const String & database, const String & table = "", + std::chrono::seconds timeout = std::chrono::seconds(60)); + + /// TODO other distributed read write lock interface can be added according to the tryLock() which is specially for database and table + + static std::shared_mutex rwlock_node_set_mutex; + static std::unordered_set rwlock_node_set; + +private: + void run(); + void createRWLockNode(); + void tryLockImpl(); + + zkutil::ZooKeeperPtr zk_cli = nullptr; + /// True for read lock, false for write lock. + bool readonly; + /// Parent path of rwlock node, eg: /clickhouse/rwlocks/database01/tables/table01 + String parent_path; + /// Current lock node, eg: /clickhouse/rwlocks/database01/tables/table01/w-0000000001 + String rwlock_node; + + std::chrono::seconds wait_timeout_seconds; + + Poco::Logger * log; + + enum class LockResult + { + INIT = 0, + SUCCESS = 1, + RETRY = 2, + FAILED = 3, + }; + std::atomic try_lock_result; + + std::mutex watch_cond_mutex; + std::condition_variable watch_cond; +}; +} diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b2cf062808a5..b115dd8e0f21 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -16,6 +16,7 @@ #include #include #include +#include #include "config_core.h" @@ -297,7 +298,9 @@ class Context: public std::enable_shared_from_this public: - DistributedDDLGuardPtr distributed_ddl_guard; + zkutil::DistributedRWLockPtr ddl_database_lock; + zkutil::DistributedRWLockPtr ddl_table_lock; + struct QueryPlanFragmentInfo { String initial_query_id; diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 8cc9e1f408e4..381a55be7774 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include @@ -37,7 +38,7 @@ namespace ErrorCodes } -InterpreterAlterQuery::InterpreterAlterQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) +InterpreterAlterQuery::InterpreterAlterQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) { } @@ -70,6 +71,11 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) && !getContext()->getClientInfo().is_replicated_database_internal && !alter.isFetchAlter()) { + if (table_id.database_name != "system") + { + getContext()->ddl_database_lock = zkutil::DistributedRWLock::tryLock(getContext()->getZooKeeper(), true, table_id.database_name); + getContext()->ddl_table_lock = zkutil::DistributedRWLock::tryLock(getContext()->getZooKeeper(), false, table_id.database_name, table_id.table_name); + } auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name); guard->releaseTableLock(); return typeid_cast(database.get())->tryEnqueueReplicatedDDL(query_ptr, getContext()); diff --git a/src/Interpreters/InterpreterAlterQuery.h b/src/Interpreters/InterpreterAlterQuery.h index 9494a400e7b3..c2d84849c467 100644 --- a/src/Interpreters/InterpreterAlterQuery.h +++ b/src/Interpreters/InterpreterAlterQuery.h @@ -15,10 +15,10 @@ class ASTAlterQuery; /** Allows you add or remove a column in the table. * It also allows you to manipulate the partitions of the MergeTree family tables. */ -class InterpreterAlterQuery : public IInterpreter, WithContext +class InterpreterAlterQuery : public IInterpreter, WithMutableContext { public: - InterpreterAlterQuery(const ASTPtr & query_ptr_, ContextPtr context_); + InterpreterAlterQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_); BlockIO execute() override; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 580fff7ee43b..12aef5634b97 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -65,6 +65,7 @@ #include #include +#include namespace DB @@ -106,10 +107,10 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) /// If the statement is not from ddl and not internal, that is from the user, the database requires distributed ddl guard if (!getContext()->getClientInfo().is_replicated_database_internal && !internal) { - getContext()->distributed_ddl_guard = DatabaseCatalog::instance().getDistributedDDLGuard(database_name); - if (!getContext()->distributed_ddl_guard->isCreated()) - throw Exception( - "Database " + database_name + " is locked by another one, couldn't acquire lock", ErrorCodes::CANNOT_CREATE_DATABASE); + if (database_name != "system") + { + getContext()->ddl_database_lock = zkutil::DistributedRWLock::tryLock(getContext()->getZooKeeper(), false, database_name); + } } /// Database can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard @@ -869,12 +870,20 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) { /// Temporary tables are created out of databases. if (create.temporary && !create.database.empty()) - throw Exception("Temporary tables cannot be inside a database. You should not specify a database for a temporary table.", + throw Exception( + "Temporary tables cannot be inside a database. You should not specify a database for a temporary table.", ErrorCodes::BAD_DATABASE_FOR_TEMPORARY_TABLE); String current_database = getContext()->getCurrentDatabase(); auto database_name = create.database.empty() ? current_database : create.database; + if (!getContext()->getClientInfo().is_replicated_database_internal && !internal && database_name != "system") + { + getContext()->ddl_database_lock = zkutil::DistributedRWLock::tryLock(getContext()->getZooKeeper(), true, database_name); + getContext()->ddl_table_lock = zkutil::DistributedRWLock::tryLock(getContext()->getZooKeeper(), false, database_name, create.table); + } + + // If this is a stub ATTACH query, read the query definition from the database if (create.attach && !create.storage && !create.columns_list) { @@ -1146,7 +1155,7 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, const InterpreterCreateQuery::TableProperties & properties) { /// Replicated database requires separate contexts for each DDL query - ContextPtr current_context = getContext(); + ContextMutablePtr current_context = getContext(); ContextMutablePtr create_context = Context::createCopy(current_context); create_context->setQueryContext(std::const_pointer_cast(current_context)); diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index b009d7046253..c79194629a38 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -13,6 +13,7 @@ #include #include "config_core.h" +#include #if USE_MYSQL # include @@ -81,6 +82,14 @@ BlockIO InterpreterDropQuery::executeToTable(ASTDropQuery & query) { DatabasePtr database; UUID table_to_wait_on = UUIDHelpers::Nil; + auto table_id = StorageID(query); + if (!getContext()->getClientInfo().is_replicated_database_internal && table_id.database_name != "system") + { + if (table_id.database_name.empty()) + query.database = table_id.database_name = getContext()->getCurrentDatabase(); + getContext()->ddl_database_lock = zkutil::DistributedRWLock::tryLock(getContext()->getZooKeeper(), true, table_id.database_name); + getContext()->ddl_table_lock = zkutil::DistributedRWLock::tryLock(getContext()->getZooKeeper(), false, table_id.database_name,table_id.table_name); + } auto res = executeToTableImpl(query, database, table_to_wait_on); if (query.no_delay) waitForTableToBeActuallyDroppedOrDetached(query, database, table_to_wait_on); @@ -308,10 +317,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, if (!getContext()->getClientInfo().is_replicated_database_internal) { - getContext()->distributed_ddl_guard = DatabaseCatalog::instance().getDistributedDDLGuard(database_name); - if (!getContext()->distributed_ddl_guard->isCreated()) - throw Exception( - "Database " + database_name + " is locked by another one, couldn't acquire lock", ErrorCodes::CANNOT_DROP_DATABASE); + getContext()->ddl_database_lock = zkutil::DistributedRWLock::tryLock(getContext()->getZooKeeper(), false, database_name); } database = tryGetDatabase(database_name, query.if_exists); diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index 5c604034ec01..54e48cf08c8a 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -18,8 +18,8 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -InterpreterRenameQuery::InterpreterRenameQuery(const ASTPtr & query_ptr_, ContextPtr context_) - : WithContext(context_), query_ptr(query_ptr_) +InterpreterRenameQuery::InterpreterRenameQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_) { } @@ -50,6 +50,7 @@ BlockIO InterpreterRenameQuery::execute() /// Don't allow to drop tables (that we are renaming); don't allow to create tables in places where tables will be renamed. TableGuards table_guards; + std::map distributed_ddl_rw_locks; for (const auto & elem : rename.elements) { descriptions.emplace_back(elem, current_database); @@ -60,6 +61,32 @@ BlockIO InterpreterRenameQuery::execute() table_guards[from]; table_guards[to]; + + /// Add distributed read-write lock for rename/exchange database/table operation + auto add_distributed_lock = [&](const String & db_name, const String & table_name, bool readonly) + { + String key = db_name; + if (!table_name.empty()) + { + /// In practical no database should contains '/', this might cause error + key += ("/" + table_name); + } + + if (!distributed_ddl_rw_locks.contains(key)) + distributed_ddl_rw_locks[key] = zkutil::DistributedRWLock::tryLock(getContext()->getZooKeeper(), readonly, db_name, table_name); + }; + + /// rename database not supported yet, so the 'rename.database' should always be false now. + /// But the code should also works for that case too + /// Add lock for both the original database/table and target database/table. + add_distributed_lock(from.database_name, "", !rename.database); + add_distributed_lock(to.database_name, "", !rename.database); + if (!rename.database) + { + add_distributed_lock(from.database_name, from.table_name, false); + if (rename.exchange) + add_distributed_lock(to.database_name, to.table_name, false); + } } auto & database_catalog = DatabaseCatalog::instance(); diff --git a/src/Interpreters/InterpreterRenameQuery.h b/src/Interpreters/InterpreterRenameQuery.h index dfcd741754ea..4a77091be7bc 100644 --- a/src/Interpreters/InterpreterRenameQuery.h +++ b/src/Interpreters/InterpreterRenameQuery.h @@ -48,10 +48,10 @@ using TableGuards = std::map>; /** Rename one table * or rename many tables at once. */ -class InterpreterRenameQuery : public IInterpreter, WithContext +class InterpreterRenameQuery : public IInterpreter, WithMutableContext { public: - InterpreterRenameQuery(const ASTPtr & query_ptr_, ContextPtr context_); + InterpreterRenameQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_); BlockIO execute() override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const override; diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index cebb3b89f8e4..1c30bbd28342 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -108,7 +108,7 @@ class StorageMaterializedPostgreSQL final : public shared_ptr_helper Date: Mon, 24 Jan 2022 20:38:53 +0800 Subject: [PATCH 71/84] Remove table system.processes from distributed query plan --- src/Processors/QueryPlan/DistributedPlanner.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/DistributedPlanner.cpp b/src/Processors/QueryPlan/DistributedPlanner.cpp index 2cd9fd78d418..90fe8d18935d 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.cpp +++ b/src/Processors/QueryPlan/DistributedPlanner.cpp @@ -373,7 +373,7 @@ bool DistributedPlanner::scheduleStages(PlanResult & plan_result) "SystemCurrentRoles", "SystemEnabledRoles", "SystemRowPolicies", "SystemPrivileges", "SystemQuotas", "SystemQuotaLimits", "SystemSettings", "SystemSettingsProfiles", "SystemSettingsProfileElements", - "SystemZooKeeper", + "SystemZooKeeper", "SystemProcesses", "SystemNumbers", "SystemOne", "SystemZeros", "SystemContributors", "SystemLicenses", "SystemReplicatedMergeTreeSettings", "SystemMergeTreeSettings"}; From 945ac8774cdb13302763383541eeff55c2f58d33 Mon Sep 17 00:00:00 2001 From: Marble Ma Date: Tue, 25 Jan 2022 20:05:40 +0800 Subject: [PATCH 72/84] Fix grpc client read buffer size limit from 4MB to unlimit --- src/Client/GRPCClient.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Client/GRPCClient.cpp b/src/Client/GRPCClient.cpp index 3bfe140c6779..8e61bb9ce6bf 100644 --- a/src/Client/GRPCClient.cpp +++ b/src/Client/GRPCClient.cpp @@ -59,7 +59,9 @@ GRPCResult GRPCClient::executePlanFragment(const GRPCQueryInfo & query_info) void GRPCClient::prepareRead(const GRPCTicket & ticket_) { ticket = ticket_; - auto ch = grpc::CreateChannel(addr, grpc::InsecureChannelCredentials()); + grpc::ChannelArguments arg; + arg.SetMaxReceiveMessageSize(-1); + auto ch = grpc::CreateCustomChannel(addr, grpc::InsecureChannelCredentials(), arg); std::shared_ptr ctx = std::make_shared(); auto stub = clickhouse::grpc::ClickHouse::NewStub(ch); auto reader = stub->FetchPlanFragmentResult(ctx.get(), ticket); From e3dbde83faabc0e8cdd3d4b5c62656571eebbdd7 Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Tue, 25 Jan 2022 10:15:02 +0800 Subject: [PATCH 73/84] Clear garbage node --- src/Interpreters/Cluster.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 1be0db35dbe2..544d50abcb0b 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -768,6 +768,8 @@ void ClustersWatcher::startup() UInt16 grpc_port = context->getServerPort("grpc_port"); String node_data = replica + ":" + toString(grpc_port); + /// When the server restarts, garbage node may remain. Clear if exists + zookeeper->tryRemove(node_path); Coordination::Requests requests; requests.emplace_back(zkutil::makeCreateRequest(node_path, node_data, zkutil::CreateMode::Ephemeral)); From 312754282d5426cfd2bedebef61c56608fa15731 Mon Sep 17 00:00:00 2001 From: madianjun Date: Wed, 26 Jan 2022 11:32:36 +0800 Subject: [PATCH 74/84] Fix reading some storage(e.g. numbers, memory) from local instead of remote --- src/Interpreters/Context.cpp | 24 ++- src/Interpreters/Context.h | 3 + src/Interpreters/ExpressionAnalyzer.cpp | 4 + src/Interpreters/GlobalSubqueriesVisitor.h | 14 ++ .../IInterpreterUnionOrSelectQuery.cpp | 2 +- .../IInterpreterUnionOrSelectQuery.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 23 ++- src/Interpreters/InterpreterSelectQuery.h | 8 +- .../InterpreterSelectWithUnionQuery.cpp | 17 +- .../InterpreterSelectWithUnionQuery.h | 2 +- .../QueryPlan/DistributedPlanner.cpp | 165 ++++++++++++++---- src/Processors/QueryPlan/DistributedPlanner.h | 14 +- src/Storages/IStorage.cpp | 2 +- 13 files changed, 222 insertions(+), 58 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d34c14205e27..34edf27255ca 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -897,6 +897,29 @@ const Block * Context::tryGetLocalScalar(const String & name) const return &it->second; } +ExternalTableHolders Context::getExternalTableHolders() const +{ + assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL); + auto lock = getLock(); + + ExternalTableHolders res; + res.insert(external_tables_mapping.begin(), external_tables_mapping.end()); + + auto query_context_ptr = query_context.lock(); + auto session_context_ptr = session_context.lock(); + if (query_context_ptr && query_context_ptr.get() != this) + { + ExternalTableHolders buf = query_context_ptr->getExternalTableHolders(); + res.insert(buf.begin(), buf.end()); + } + else if (session_context_ptr && session_context_ptr.get() != this) + { + ExternalTableHolders buf = session_context_ptr->getExternalTableHolders(); + res.insert(buf.begin(), buf.end()); + } + return res; +} + Tables Context::getExternalTables() const { assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL); @@ -921,7 +944,6 @@ Tables Context::getExternalTables() const return res; } - void Context::addExternalTable(const String & table_name, TemporaryTableHolder && temporary_table) { assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b115dd8e0f21..39540a5a4143 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -150,6 +150,8 @@ using InputBlocksReader = std::function; /// Used in distributed task processing using ReadTaskCallback = std::function; +using ExternalTableHolders = std::unordered_map>; + /// An empty interface for an arbitrary object that may be attached by a shared pointer /// to query context, when using ClickHouse as a library. struct IHostContext @@ -482,6 +484,7 @@ class Context: public std::enable_shared_from_this StorageID tryResolveStorageID(StorageID storage_id, StorageNamespace where = StorageNamespace::ResolveAll) const; StorageID resolveStorageIDImpl(StorageID storage_id, StorageNamespace where, std::optional * exception) const; + ExternalTableHolders getExternalTableHolders() const; Tables getExternalTables() const; void addExternalTable(const String & table_name, TemporaryTableHolder && temporary_table); std::shared_ptr removeExternalTable(const String & table_name); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 820744cbdc41..6b04eff16194 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -411,6 +411,10 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global) { if (do_global) { + /// Only for debug. +// WriteBufferFromOwnString wb; +// dumpAST(*query, wb); +// LOG_DEBUG(&Poco::Logger::get("initGlobalSubqueriesAndExternalTables"), "query AST:\n{}", wb.str()); GlobalSubqueriesVisitor::Data subqueries_data(getContext(), subquery_depth, shouldAddExternalStorage(), external_tables, subqueries_for_sets, has_global_subqueries); GlobalSubqueriesVisitor(subqueries_data).visit(query); diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index b7eb54cb6a1d..e42d7f1a0f9e 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -233,6 +233,20 @@ class GlobalSubqueriesMatcher && (table_elem.table_join->as().locality == ASTTableJoin::Locality::Global || data.getContext()->getSettingsRef().prefer_global_in_and_join)) { + if (const auto * ast_table_expr = table_elem.table_expression->as()) + { + if (ast_table_expr->table_function) + { + /// For example: select * from demo, numbers(10) n. + /// TablesInSelectQueryElement (children 2) + /// TableJoin + /// TableExpression (children 1) + /// Function numbers (alias n) (children 1) # here is table_function + /// ExpressionList (children 1) + /// Literal UInt64_10 + return; + } + } data.addExternalStorage(table_elem.table_expression, true); data.has_global_subqueries = true; } diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index e718ff8ba4a0..bf99fd906543 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -14,7 +14,7 @@ void IInterpreterUnionOrSelectQuery::extendQueryLogElemImpl(QueryLogElement & el elem.query_kind = "Select"; } -void IInterpreterUnionOrSelectQuery::rewriteDistributedQuery(bool is_subquery, [[maybe_unused]] size_t tables_count) +void IInterpreterUnionOrSelectQuery::rewriteDistributedQuery(bool is_subquery, [[maybe_unused]] size_t tables_count, bool) { /// Reset these fields to not share them with parent query. if (is_subquery) diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index f4342dd6e3f6..5af88b48ef8a 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -31,7 +31,7 @@ class IInterpreterUnionOrSelectQuery : public IInterpreter context->getClientInfo().distributed_query = queryToString(query_ptr); } - virtual void rewriteDistributedQuery(bool is_subquery, size_t tables_count = 0); + virtual void rewriteDistributedQuery(bool is_subquery, size_t tables_count = 0, bool need_log = false); virtual void buildQueryPlan(QueryPlan & query_plan) = 0; QueryPipelineBuilder buildQueryPipeline(); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a8a8d52ac7ab..b002077d8c6a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -582,7 +582,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( sanitizeBlock(result_header, true); } -void InterpreterSelectQuery::rewriteDistributedQuery(bool is_subquery, size_t tables_count) +void InterpreterSelectQuery::rewriteDistributedQuery(bool is_subquery, size_t tables_count, bool need_log) { IInterpreterUnionOrSelectQuery::rewriteDistributedQuery(is_subquery); @@ -593,12 +593,13 @@ void InterpreterSelectQuery::rewriteDistributedQuery(bool is_subquery, size_t ta } String maybe_rewritten_query = queryToString(distributed_query_ptr); - LOG_DEBUG( - log, - "[{}] Rewrite\n\"{}\"\n=>\n\"{}\"", - static_cast(context.get()), - context->getClientInfo().distributed_query, - maybe_rewritten_query); + if (need_log) + LOG_DEBUG( + log, + "[{}] Rewrite\n\"{}\"\n=>\n\"{}\"", + static_cast(context.get()), + context->getClientInfo().distributed_query, + maybe_rewritten_query); context->getClientInfo().distributed_query = std::move(maybe_rewritten_query); } @@ -1519,7 +1520,11 @@ static void executeMergeAggregatedImpl( } void InterpreterSelectQuery::addEmptySourceToQueryPlan( - QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, ContextPtr context_) + QueryPlan & query_plan, + const Block & source_header, + const SelectQueryInfo & query_info, + ContextPtr context_, + const String & storage_name) { Pipe pipe(std::make_shared(source_header)); @@ -1556,7 +1561,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan( } auto read_from_pipe = std::make_unique(std::move(pipe)); - read_from_pipe->setStepDescription("Read from NullSource"); + read_from_pipe->setStepDescription(storage_name.empty() ? "Read from NullSource" : storage_name); InterpreterParamsPtr interpreter_params = std::make_shared(context_, query_info.query->as()); query_plan.addStep(std::move(read_from_pipe), std::move(interpreter_params)); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 075365158793..85a4705fa006 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -76,7 +76,7 @@ class InterpreterSelectQuery : public IInterpreterUnionOrSelectQuery ~InterpreterSelectQuery() override; - virtual void rewriteDistributedQuery(bool is_subquery, size_t tables_count) override; + virtual void rewriteDistributedQuery(bool is_subquery, size_t tables_count, bool need_log = false) override; /// Execute a query. Get the stream of blocks to read. BlockIO execute() override; @@ -102,7 +102,11 @@ class InterpreterSelectQuery : public IInterpreterUnionOrSelectQuery bool hasAggregation() const { return query_analyzer->hasAggregation(); } static void addEmptySourceToQueryPlan( - QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, ContextPtr context_); + QueryPlan & query_plan, + const Block & source_header, + const SelectQueryInfo & query_info, + ContextPtr context_, + const String & storage_name = ""); Names getRequiredColumns() { return required_columns; } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 638de4d84172..4357176f673e 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -169,7 +169,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( options.ignore_quota |= all_nested_ignore_quota; } -void InterpreterSelectWithUnionQuery::rewriteDistributedQuery(bool is_subquery, size_t) +void InterpreterSelectWithUnionQuery::rewriteDistributedQuery(bool is_subquery, size_t, bool need_log) { IInterpreterUnionOrSelectQuery::rewriteDistributedQuery(is_subquery); @@ -182,12 +182,13 @@ void InterpreterSelectWithUnionQuery::rewriteDistributedQuery(bool is_subquery, distributed_ast->list_of_selects->children.at(i) = nested_interpreters[i]->getDistributedQueryPtr()->clone(); rewritten_query = queryToString(distributed_query_ptr); - LOG_DEBUG( - log, - "[{}] Rewrite\n\"{}\"\n=>\n\"{}\"", - static_cast(context.get()), - context->getClientInfo().distributed_query, - rewritten_query); + if (need_log) + LOG_DEBUG( + log, + "[{}] Rewrite\n\"{}\"\n=>\n\"{}\"", + static_cast(context.get()), + context->getClientInfo().distributed_query, + rewritten_query); context->getClientInfo().distributed_query = std::move(rewritten_query); } @@ -353,7 +354,7 @@ BlockIO InterpreterSelectWithUnionQuery::execute() QueryPlan query_plan; buildQueryPlan(query_plan); - rewriteDistributedQuery(false); + rewriteDistributedQuery(false, 0, true); query_plan.checkInitialized(); query_plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index 5839e54fabb3..9a1af5596398 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -24,7 +24,7 @@ class InterpreterSelectWithUnionQuery : public IInterpreterUnionOrSelectQuery ~InterpreterSelectWithUnionQuery() override; - virtual void rewriteDistributedQuery(bool is_subquery, size_t tables_count = 0) override; + virtual void rewriteDistributedQuery(bool is_subquery, size_t tables_count = 0, bool need_log = false) override; /// Builds QueryPlan for current query. virtual void buildQueryPlan(QueryPlan & query_plan) override; diff --git a/src/Processors/QueryPlan/DistributedPlanner.cpp b/src/Processors/QueryPlan/DistributedPlanner.cpp index 90fe8d18935d..7496cc441e4a 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.cpp +++ b/src/Processors/QueryPlan/DistributedPlanner.cpp @@ -26,6 +26,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int TABLE_ALREADY_EXISTS; } DistributedPlanner::DistributedPlanner(QueryPlan & query_plan_, const ContextMutablePtr & context_) @@ -39,7 +40,8 @@ void DistributedPlanner::checkShuffle( QueryPlan::Node * current_node, QueryPlan::Node * child_node, CheckShuffleResult & result, - StageSeq & stage_seq) + StageSeq & stage_seq, + std::stack & leaf_nodes) { /// Cases requiring special attention: /// 1. Distinct: @@ -71,7 +73,19 @@ void DistributedPlanner::checkShuffle( assert(current_node->children.size() == 2); /// Only broadcast right side. if (child_node == current_node->children[1]) - result.is_shuffle = true; + { + if (child_node->num_parent_stages == 0 && child_node->num_leaf_nodes_in_stage == 1 + && !leaf_nodes.empty() && isSinglePointDataSource(leaf_nodes.top()->step->getStepDescription())) + { + /// 1. Storage name is SystemNumbers: select ... from t1, numbers(10) n + /// 2. Storage name is Memory: select ... from t1 join (select number from numbers(1)) + /// 3. Storage name is Memory: select ... from t1 join t2 + LOG_DEBUG(log, "No need shuffle: right child's storage is {}", leaf_nodes.top()->step->getStepDescription()); + result.is_shuffle = false; + } + else + result.is_shuffle = true; + } stage_seq = StageSeq::STAGE1; return; } @@ -187,7 +201,7 @@ void DistributedPlanner::buildStages() { /// This is shuffle, create a new stage for child_node. CheckShuffleResult result; - checkShuffle(frame.node, last_node, result, stage_seq); + checkShuffle(frame.node, last_node, result, stage_seq, leaf_nodes); if (result.is_shuffle) { ++stage_id; @@ -246,7 +260,7 @@ void DistributedPlanner::buildStages() if (next_child == frame.node->children.size() || (dynamic_cast(frame.node->step.get()) && frame.visited_children == 1)) { - LOG_DEBUG(log, "[{}]Visit step: {}", getStageSeqName(stage_seq), frame.node->step->getName()); + LOG_DEBUG(log, "[{}]Visit step: {}({})", getStageSeqName(stage_seq), frame.node->step->getName(), frame.node->step->getStepDescription()); last_node = frame.node; one_child_is_visited = true; stack.pop(); @@ -264,10 +278,23 @@ void DistributedPlanner::buildStages() /// Create a virtual node, used in iterating stages. parent_stages.push(last_stage); auto step = std::make_unique(stage_id, last_stage->id, context); - query_plan.nodes.emplace_back(QueryPlan::Node{.step = std::move(step), .children = {last_node}, .num_parent_stages = 1}); + query_plan.nodes.emplace_back(QueryPlan::Node{ + .step = std::move(step), + .children = {last_node}, + .num_parent_stages = 1, + .interpreter_params = query_plan.root->interpreter_params}); query_plan.root = &query_plan.nodes.back(); + LOG_DEBUG( + log, + "Set context({} <= {}) to {}", + query_plan.root->step->getName(), + last_node->step->getName(), + static_cast(query_plan.root->interpreter_params->context.get())); + + /// Maintain leaf nodes. leaf_nodes.push(query_plan.root); query_plan.root->num_leaf_nodes_in_stage = 1; + result_stage = createStage(stage_id, parent_stages, query_plan.root, leaf_nodes); debugStages(); @@ -328,7 +355,25 @@ void DistributedPlanner::debugStages() LOG_DEBUG(log, "===> Print Stages:\n{}", buf.str()); } -bool DistributedPlanner::scheduleStages(PlanResult & plan_result) +bool DistributedPlanner::isSinglePointDataSource(const String & name) +{ + /// They are ReadFromStorageStep with storage name as follows. + static std::unordered_set single_point_storages{"SystemClusters", "SystemDatabases", "SystemTables", "SystemColumns", + "SystemDictionaries", "SystemDataSkippingIndices", + "SystemFunctions", "SystemFormats", "SystemTableEngines", + "SystemUsers", "SystemRoles", "SystemGrants", "SystemRoleGrants", + "SystemCurrentRoles", "SystemEnabledRoles", "SystemRowPolicies", "SystemPrivileges", + "SystemQuotas", "SystemQuotaLimits", + "SystemSettings", "SystemSettingsProfiles", "SystemSettingsProfileElements", + "SystemZooKeeper", "SystemProcesses", + "SystemNumbers", "SystemOne", "SystemZeros", + "SystemContributors", "SystemLicenses", + "SystemReplicatedMergeTreeSettings", "SystemMergeTreeSettings", + "Memory"}; + return single_point_storages.contains(name); +} + +void DistributedPlanner::scheduleStages(PlanResult & plan_result) { LOG_DEBUG(log, "===> Schedule stages."); /// Use current query id to build the plan fragment id. @@ -366,21 +411,8 @@ bool DistributedPlanner::scheduleStages(PlanResult & plan_result) if (store_replicas.empty() || compute_replicas.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "No enough store workers({}) or compute workers({}).", store_replicas.size(), compute_replicas.size()); - static std::unordered_set system_tables{"SystemClusters", "SystemDatabases", "SystemTables", "SystemColumns", - "SystemDictionaries", "SystemDataSkippingIndices", - "SystemFunctions", "SystemFormats", "SystemTableEngines", - "SystemUsers", "SystemRoles", "SystemGrants", "SystemRoleGrants", - "SystemCurrentRoles", "SystemEnabledRoles", "SystemRowPolicies", "SystemPrivileges", - "SystemQuotas", "SystemQuotaLimits", - "SystemSettings", "SystemSettingsProfiles", "SystemSettingsProfileElements", - "SystemZooKeeper", "SystemProcesses", - "SystemNumbers", "SystemOne", "SystemZeros", - "SystemContributors", "SystemLicenses", - "SystemReplicatedMergeTreeSettings", "SystemMergeTreeSettings"}; - static std::unordered_set special_storages{"HDFS", "S3", "MySQL", "Memory"}; - bool is_result_stage_moved_forward = false; auto fillStage = [&](Stage * stage) { /// Leaf stage. @@ -393,11 +425,11 @@ bool DistributedPlanner::scheduleStages(PlanResult & plan_result) if (leaf_node->children.empty()) { /// It's system table or special storage. - if (system_tables.contains(leaf_node->step->getStepDescription()) || + if (isSinglePointDataSource(leaf_node->step->getStepDescription()) || special_storages.contains(leaf_node->step->getStepDescription())) { } - /// It's StorageValues. + /// It's StorageValues. else if (leaf_node->step->getStepDescription() == "Values") { /// StorageValues is used in: @@ -454,7 +486,7 @@ bool DistributedPlanner::scheduleStages(PlanResult & plan_result) result_stage = parent; query_plan.root = parent->root_node; stages.pop_back(); - is_result_stage_moved_forward = true; + plan_result.is_result_stage_moved_forward = true; return; } } @@ -530,7 +562,7 @@ bool DistributedPlanner::scheduleStages(PlanResult & plan_result) if (&stage == result_stage) { assert(!result_stage->parents.empty()); - if (is_result_stage_moved_forward) + if (plan_result.is_result_stage_moved_forward) { Context::QueryPlanFragmentInfo query_plan_fragment_info{ .initial_query_id = initial_query_id, @@ -543,6 +575,36 @@ bool DistributedPlanner::scheduleStages(PlanResult & plan_result) } query_plan_fragment_info.sinks = stage.sinks; context->setQueryPlanFragmentInfo(query_plan_fragment_info); + + /// Add external tables to current context. + assert(stage.root_node->interpreter_params); + const auto & external_table_holders = stage.root_node->interpreter_params->context->getExternalTableHolders(); + LOG_DEBUG( + log, + "Add {} external tables using context {} from local.", + external_table_holders.size(), + static_cast(stage.root_node->interpreter_params->context.get())); + for (const auto & table : external_table_holders) + { + auto table_holder = table.second; + String table_name = table.first; + + LOG_DEBUG( + log, + "Add external table {} with {} ({}) from local.", + table_name, + table_holder->getTable()->getStorageID().getFullNameNotQuoted(), + table_holder->getTable()->getName()); + try + { + context->addExternalTable(table_name, std::move(*table_holder)); + } + catch (Exception & e) + { + if (e.code() != ErrorCodes::TABLE_ALREADY_EXISTS) + throw; + } + } } else { @@ -783,8 +845,6 @@ bool DistributedPlanner::scheduleStages(PlanResult & plan_result) : "Exception: (code " + toString(result.exception().code()) + ") " + result.exception().display_text()); } } - - return is_result_stage_moved_forward; } void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) @@ -793,14 +853,35 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) int my_stage_id = query_distributed_plan_info.stage_id; LOG_DEBUG( log, - "===> Build plan fragment: {} stage {}, has {} parent stages.", + "===> [{}]Build plan fragment: {} stage {}({} parent stages).", + plan_result.is_result_stage_moved_forward ? "From Local" : "From Remote", (result_stage ? (my_stage_id == result_stage->id ? "result": "non-result") : "non-result"), my_stage_id, query_distributed_plan_info.parent_sources.size()); + /// Clean states that may be set in building stages. + if (plan_result.is_result_stage_moved_forward) + { + for (auto & node : query_plan.nodes) + { + node.num_parent_stages = 0; + node.num_leaf_nodes_in_stage = 0; + } + } + /// Get my replica grpc address String my_replica = context->getMacros()->getValue("replica") + ":" + toString(context->getServerPort("grpc_port")); + auto popLeafNodes = [](QueryPlan::Node * root_node, std::stack & leaf_nodes) { + if (root_node) + { + for (int i = 0; !leaf_nodes.empty() && i < root_node->num_leaf_nodes_in_stage; ++i) + { + leaf_nodes.pop(); + } + } + }; + struct Frame { QueryPlan::Node * node = {}; @@ -811,10 +892,12 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) std::stack stack; stack.push(Frame{.node = query_plan.root}); bool one_child_is_visited = false; + std::stack leaf_nodes; /// Used for locating the plan fragment. int stage_id = -1; QueryPlan::Node * last_node = nullptr; + QueryPlan::Node * leaf_node = nullptr; StageSeq stage_seq = StageSeq::STAGE1; while (!stack.empty()) @@ -836,12 +919,13 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) } CheckShuffleResult result; - checkShuffle(frame.node, last_node, result, stage_seq); + checkShuffle(frame.node, last_node, result, stage_seq, leaf_nodes); /// This is a shuffle dependency between current node and the last visited child. if (result.is_shuffle) { ++stage_id; + popLeafNodes(last_node, leaf_nodes); /// This is one of my parent stages. const auto & it = query_distributed_plan_info.parent_sources.find(stage_id); @@ -980,17 +1064,36 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) return; } + + /// Set the number of parent stages and leaf nodes. + frame.node->num_parent_stages += 1; + leaf_node = frame.node; + leaf_nodes.push(leaf_node); + frame.node->num_leaf_nodes_in_stage += 1; + } + else + { + frame.node->num_parent_stages += last_node->num_parent_stages; + frame.node->num_leaf_nodes_in_stage += last_node->num_leaf_nodes_in_stage; } ++frame.visited_children; one_child_is_visited = false; } + /// Set the number of leaf nodes. + if (frame.node->children.empty()) + { + leaf_node = frame.node; + leaf_nodes.push(leaf_node); + frame.node->num_leaf_nodes_in_stage = 1; + } + size_t next_child = frame.visited_children; if (next_child == frame.node->children.size() || (dynamic_cast(frame.node->step.get()) && frame.visited_children == 1)) { - LOG_DEBUG(log, "[{}]Visit step: {}", getStageSeqName(stage_seq), frame.node->step->getName()); + LOG_DEBUG(log, "[{}]Visit step: {}({})", getStageSeqName(stage_seq), frame.node->step->getName(), frame.node->step->getStepDescription()); last_node = frame.node; one_child_is_visited = true; stack.pop(); @@ -1050,8 +1153,8 @@ bool DistributedPlanner::buildDistributedPlan() { buildStages(); - bool is_result_stage_moved_forward = scheduleStages(plan_result); - if (is_result_stage_moved_forward) + scheduleStages(plan_result); + if (plan_result.is_result_stage_moved_forward) buildPlanFragment(plan_result); if (!creating_set_plans.empty() && !dynamic_cast(query_plan.root->step.get())) @@ -1061,7 +1164,7 @@ bool DistributedPlanner::buildDistributedPlan() result_stage->root_node = query_plan.root; } - if (!is_result_stage_moved_forward) + if (!plan_result.is_result_stage_moved_forward) { plan_result.initial_query_id = context->getClientInfo().current_query_id; plan_result.stage_id = result_stage->id; diff --git a/src/Processors/QueryPlan/DistributedPlanner.h b/src/Processors/QueryPlan/DistributedPlanner.h index 3a4204048ada..3e8d4cd97845 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.h +++ b/src/Processors/QueryPlan/DistributedPlanner.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -84,7 +85,12 @@ class DistributedPlanner { LimitStep * child_limit_step = nullptr; IQueryPlanStep * grandchild_step = nullptr; }; - void checkShuffle(QueryPlan::Node * current_node, QueryPlan::Node * child_node, CheckShuffleResult & result, StageSeq & stage_seq); + void checkShuffle( + QueryPlan::Node * current_node, + QueryPlan::Node * child_node, + CheckShuffleResult & result, + StageSeq & stage_seq, + std::stack & leaf_nodes); struct PlanResult { @@ -92,14 +98,16 @@ class DistributedPlanner { int stage_id; String node_id; std::vector distributed_source_nodes; + bool is_result_stage_moved_forward = false; }; String debugLocalPlanFragment(PlanResult & plan_result); String debugRemotePlanFragment(const String & query, const String & receiver, const String & query_id, const Stage * stage); + bool isSinglePointDataSource(const String & name); void buildStages(); void debugStages(); - /// Return true if result stage is moved forward. - bool scheduleStages(PlanResult & plan_result); + + void scheduleStages(PlanResult & plan_result); void buildPlanFragment(PlanResult & plan_result); void uniteCreatingSetSteps(std::vector> & creating_set_plans); diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index ddf6fa53aad3..4733483beb41 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -115,7 +115,7 @@ void IStorage::read( { auto header = (query_info.projection ? query_info.projection->desc->metadata : metadata_snapshot) ->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); - InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, context); + InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, context, getName()); } else { From 052d008e080c62f741e966d25d9c072621cf9aaf Mon Sep 17 00:00:00 2001 From: madianjun Date: Wed, 26 Jan 2022 09:47:32 +0800 Subject: [PATCH 75/84] Eliminate unnecessary stages for sort and limit when shuffle has been created at aggregate --- .../QueryPlan/DistributedPlanner.cpp | 40 ++++++++++++------- 1 file changed, 25 insertions(+), 15 deletions(-) diff --git a/src/Processors/QueryPlan/DistributedPlanner.cpp b/src/Processors/QueryPlan/DistributedPlanner.cpp index 7496cc441e4a..a9e011f60de4 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.cpp +++ b/src/Processors/QueryPlan/DistributedPlanner.cpp @@ -60,7 +60,7 @@ void DistributedPlanner::checkShuffle( result.current_union_step = dynamic_cast(current_node->step.get()); if (result.current_union_step) { - LOG_DEBUG(log, "Check shuffle: child node is UnionStep"); + LOG_DEBUG(log, "[{}]Check shuffle: child node is UnionStep", getStageSeqName(stage_seq)); result.is_shuffle = true; stage_seq = StageSeq::STAGE1; return; @@ -69,7 +69,7 @@ void DistributedPlanner::checkShuffle( result.current_join_step = dynamic_cast(current_node->step.get()); if (result.current_join_step) { - LOG_DEBUG(log, "Check shuffle: current node is JoinStep(0x{})", static_cast(result.current_join_step)); + LOG_DEBUG(log, "[{}]Check shuffle: current node is JoinStep(0x{})", getStageSeqName(stage_seq), static_cast(result.current_join_step)); assert(current_node->children.size() == 2); /// Only broadcast right side. if (child_node == current_node->children[1]) @@ -95,7 +95,7 @@ void DistributedPlanner::checkShuffle( { /// From : AggregatingStep => /// To : AggregatingStep(partial) [=> MergingAggregatedStep(final)] => - LOG_DEBUG(log, "Check shuffle: child node is AggregatingStep"); + LOG_DEBUG(log, "[{}]Check shuffle: child node is AggregatingStep", getStageSeqName(stage_seq)); result.grandchild_step = child_node->children.front()->step.get(); result.is_shuffle = true; stage_seq = StageSeq::STAGE2; @@ -107,9 +107,12 @@ void DistributedPlanner::checkShuffle( { /// From : SortingStep => Not (LimitStep) => /// To : SortingStep(partial) [=> SortingStep(final)] => Not (LimitStep) => - LOG_DEBUG(log, "Check shuffle: child node is SortingStep"); - result.current_limit_step = dynamic_cast(current_node->step.get()); - if (!result.current_limit_step) + LOG_DEBUG(log, "[{}]Check shuffle: child node is SortingStep", getStageSeqName(stage_seq)); + if (stage_seq == StageSeq::STAGE2) + { + result.is_shuffle = false; + } + else if (!(result.current_limit_step = dynamic_cast(current_node->step.get()))) { result.is_shuffle = true; stage_seq = StageSeq::STAGE2; @@ -119,7 +122,7 @@ void DistributedPlanner::checkShuffle( if ((result.child_distinct_step = dynamic_cast(child_node->step.get()))) { - LOG_DEBUG(log, "Check shuffle: child node is DistinctStep"); + LOG_DEBUG(log, "[{}]Check shuffle: child node is DistinctStep", getStageSeqName(stage_seq)); result.current_distinct_step = dynamic_cast(current_node->step.get()); if (result.current_distinct_step) { @@ -132,14 +135,21 @@ void DistributedPlanner::checkShuffle( if ((result.child_limit_step = dynamic_cast(child_node->step.get()))) { - LOG_DEBUG(log, "Check shuffle: child node is LimitStep"); - assert(child_node->children.size() == 1); - result.grandchild_sorting_step = dynamic_cast(child_node->children[0]->step.get()); - /// If grandchild is SortingStep: - /// From : SortingStep => LimitStep => - /// To : SortingStep(partial) => LimitStep(partial) [=> SortingStep(final) => LimitStep(final)] => - result.is_shuffle = true; - stage_seq = StageSeq::STAGE2; + LOG_DEBUG(log, "[{}]Check shuffle: child node is LimitStep", getStageSeqName(stage_seq)); + if (stage_seq == StageSeq::STAGE2) + { + result.is_shuffle = false; + } + else + { + assert(child_node->children.size() == 1); + result.grandchild_sorting_step = dynamic_cast(child_node->children[0]->step.get()); + /// If grandchild is SortingStep: + /// From : SortingStep => LimitStep => + /// To : SortingStep(partial) => LimitStep(partial) [=> SortingStep(final) => LimitStep(final)] => + result.is_shuffle = true; + stage_seq = StageSeq::STAGE2; + } return; } } From 1b88b4d04e7977a02e268de67e1e5a2d07739cd3 Mon Sep 17 00:00:00 2001 From: Allen Zhang Date: Tue, 25 Jan 2022 20:17:19 +0800 Subject: [PATCH 76/84] Add default database name for SYSTEM --- src/Interpreters/InterpreterSystemQuery.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 5de840b12e25..aee14d523867 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -206,6 +206,10 @@ InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, Contex BlockIO InterpreterSystemQuery::execute() { auto & query = query_ptr->as(); + if (query.database.empty()) + { + query.database = getContext()->getCurrentDatabase(); + } using Type = ASTSystemQuery::Type; From 8b0fc4cd2b1151ea202005c78747ec22c4f3f092 Mon Sep 17 00:00:00 2001 From: caspian Date: Wed, 26 Jan 2022 11:01:13 +0800 Subject: [PATCH 77/84] Fix ddl timeout when other ck should fail fast --- src/Interpreters/DDLTask.h | 6 ++++++ src/Interpreters/DDLWorker.cpp | 14 +++++++++++++- 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index 3bae37ceb8cb..c07911cc1782 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -200,6 +200,12 @@ class ZooKeeperMetadataTransaction ops.emplace_back(op); } + void removeLastOp() + { + if (ops.size() > 0) + ops.resize(ops.size()-1); + } + void moveOpsTo(Coordination::Requests & other_ops) { if (isExecuted()) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index d7984af30c72..9d4f4cd14df2 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -514,9 +514,9 @@ bool DDLWorker::tryExecuteQuery(const String & query, DDLTaskBase & task, const WriteBufferFromString ostr(dummy_string); std::optional query_scope; + auto query_context = task.makeQueryContext(context, zookeeper); try { - auto query_context = task.makeQueryContext(context, zookeeper); if (!task.is_initial_query) query_scope.emplace(query_context); executeQuery(istr, ostr, !task.is_initial_query, query_context, {}); @@ -537,6 +537,18 @@ bool DDLWorker::tryExecuteQuery(const String & query, DDLTaskBase & task, const task.execution_status = ExecutionStatus::fromCurrentException(); tryLogCurrentException(log, "Query " + query + " wasn't finished successfully"); + /// Write fail status back to zookeeper immediately + if (auto txn = query_context->getZooKeeperMetadataTransaction()) + { + if (!txn->isExecuted()) + { + /// Remove the last zookeeper operation which is success status and write fail status back to zookeeper + txn->removeLastOp(); + txn->addOp(zkutil::makeCreateRequest(task.getFinishedNodePath(), ExecutionStatus(task.execution_status).serializeText(), zkutil::CreateMode::Persistent)); + txn->commit(); + } + } + /// We use return value of tryExecuteQuery(...) in tryExecuteQueryOnLeaderReplica(...) to determine /// if replica has stopped being leader and we should retry query. /// However, for the majority of exceptions there is no sense to retry, because most likely we will just From fdbae38cce75ffa21ccf66a3c42b15463a75e131 Mon Sep 17 00:00:00 2001 From: madianjun Date: Thu, 27 Jan 2022 09:59:33 +0800 Subject: [PATCH 78/84] Change parent stages order to sequential in order that iterating stages is from left branch to right branch --- .../QueryPlan/DistributedPlanner.cpp | 20 ++++++++++++------- src/Processors/QueryPlan/QueryPlan.h | 4 ++-- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/DistributedPlanner.cpp b/src/Processors/QueryPlan/DistributedPlanner.cpp index a9e011f60de4..8ce2f2822afd 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.cpp +++ b/src/Processors/QueryPlan/DistributedPlanner.cpp @@ -164,17 +164,23 @@ void DistributedPlanner::buildStages() if (root_node) { - for (int i = 0; !parent_stages.empty() && i < root_node->num_parent_stages; ++i) + /// Fill parent stages. + assert(parent_stages.size() >= root_node->num_parent_stages); + new_stage->parents.resize(root_node->num_parent_stages); + for (int i = root_node->num_parent_stages - 1; !parent_stages.empty() && i >= 0; --i) { - new_stage->parents.emplace_back(parent_stages.top()); - parent_stages.top()->child = new_stage; + new_stage->parents[i] = parent_stages.top(); + new_stage->parents[i]->child = new_stage; parent_stages.pop(); } - for (int i = 0; !leaf_nodes.empty() && i < root_node->num_leaf_nodes_in_stage; ++i) + /// Fill leaf nodes. + assert(leaf_nodes.size() >= root_node->num_leaf_nodes_in_stage); + new_stage->leaf_nodes.resize(root_node->num_leaf_nodes_in_stage); + for (int i = root_node->num_leaf_nodes_in_stage - 1; !leaf_nodes.empty() && i >= 0; --i) { - new_stage->leaf_nodes.emplace_back(leaf_nodes.top()); + new_stage->leaf_nodes[i] = leaf_nodes.top(); /// This leaf node is a data source node reading data from storage. - if (leaf_nodes.top()->children.empty()) + if (new_stage->leaf_nodes[i]->children.empty()) new_stage->is_leaf_stage = true; leaf_nodes.pop(); } @@ -885,7 +891,7 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) auto popLeafNodes = [](QueryPlan::Node * root_node, std::stack & leaf_nodes) { if (root_node) { - for (int i = 0; !leaf_nodes.empty() && i < root_node->num_leaf_nodes_in_stage; ++i) + for (size_t i = 0; !leaf_nodes.empty() && i < root_node->num_leaf_nodes_in_stage; ++i) { leaf_nodes.pop(); } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index b141a763bbff..03d17171ca4c 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -124,8 +124,8 @@ class QueryPlan QueryPlanStepPtr step; std::vector children = {}; Node * parent = nullptr; - int num_parent_stages = 0; /// Number of parent stages whose child is the stage current node belongs to. - int num_leaf_nodes_in_stage = 0; /// Number of leaf nodes(including current node and its descendant nodes) in the same stage. + size_t num_parent_stages = 0; /// Number of parent stages whose child is the stage current node belongs to. + size_t num_leaf_nodes_in_stage = 0; /// Number of leaf nodes(including current node and its descendant nodes) in the same stage. InterpreterParamsPtr interpreter_params; }; From 5ad6f435c5aad7ee319cf7f6bf61b56d648861a0 Mon Sep 17 00:00:00 2001 From: madianjun Date: Thu, 27 Jan 2022 10:33:58 +0800 Subject: [PATCH 79/84] Set current query id if empty --- src/Interpreters/IInterpreterUnionOrSelectQuery.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index bf99fd906543..09102eb3a707 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -23,6 +23,10 @@ void IInterpreterUnionOrSelectQuery::rewriteDistributedQuery(bool is_subquery, [ getContext()->getClientInfo().query_kind = ClientInfo::QueryKind::INITIAL_QUERY; getContext()->getClientInfo().current_query_id = getContext()->generateQueryId(); } + + /// Set current query id if empty, because distributed plan needs it as initial query id. + if (getContext()->getClientInfo().current_query_id.empty()) + getContext()->getClientInfo().current_query_id = getContext()->generateQueryId(); } QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline() From e0ee52c524b87d781869864b6cab16b6fcb1932c Mon Sep 17 00:00:00 2001 From: caspian Date: Fri, 28 Jan 2022 08:54:03 +0000 Subject: [PATCH 80/84] Fix RENAME query when using distributed lock --- src/Interpreters/InterpreterCreateQuery.cpp | 1 + src/Interpreters/InterpreterRenameQuery.cpp | 47 ++++++++++++--------- 2 files changed, 27 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 12aef5634b97..a0095b57e05f 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1227,6 +1227,7 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, ASTRenameQuery::Table{create.database, table_to_replace_name} }; + ast_rename->is_initial = false; ast_rename->elements.push_back(std::move(elem)); ast_rename->dictionary = create.is_dictionary; if (create.create_or_replace) diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index 54e48cf08c8a..45cbb902db67 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -51,6 +52,20 @@ BlockIO InterpreterRenameQuery::execute() TableGuards table_guards; std::map distributed_ddl_rw_locks; + /// Add distributed read-write lock for rename/exchange database/table operation + auto add_distributed_lock = [&](const String & db_name, const String & table_name, bool readonly) + { + String key = db_name; + if (!table_name.empty()) + { + /// In practical no database should contains '/', this might cause error + key += ("/" + table_name); + } + + if (!distributed_ddl_rw_locks.contains(key)) + distributed_ddl_rw_locks[key] = zkutil::DistributedRWLock::tryLock(getContext()->getZooKeeper(), readonly, db_name, table_name); + }; + for (const auto & elem : rename.elements) { descriptions.emplace_back(elem, current_database); @@ -62,30 +77,20 @@ BlockIO InterpreterRenameQuery::execute() table_guards[from]; table_guards[to]; - /// Add distributed read-write lock for rename/exchange database/table operation - auto add_distributed_lock = [&](const String & db_name, const String & table_name, bool readonly) + /// Only add distributed lock when it's initial query + if (auto txn = getContext()->getZooKeeperMetadataTransaction() && rename.is_initial) { - String key = db_name; - if (!table_name.empty()) + /// Rename database not supported yet, so the 'rename.database' should always be false now. + /// But the code should also works for that case too + /// Add lock for both the original database/table and target database/table. + add_distributed_lock(from.database_name, "", !rename.database); + add_distributed_lock(to.database_name, "", !rename.database); + if (!rename.database) { - /// In practical no database should contains '/', this might cause error - key += ("/" + table_name); + add_distributed_lock(from.database_name, from.table_name, false); + if (rename.exchange) + add_distributed_lock(to.database_name, to.table_name, false); } - - if (!distributed_ddl_rw_locks.contains(key)) - distributed_ddl_rw_locks[key] = zkutil::DistributedRWLock::tryLock(getContext()->getZooKeeper(), readonly, db_name, table_name); - }; - - /// rename database not supported yet, so the 'rename.database' should always be false now. - /// But the code should also works for that case too - /// Add lock for both the original database/table and target database/table. - add_distributed_lock(from.database_name, "", !rename.database); - add_distributed_lock(to.database_name, "", !rename.database); - if (!rename.database) - { - add_distributed_lock(from.database_name, from.table_name, false); - if (rename.exchange) - add_distributed_lock(to.database_name, to.table_name, false); } } From 0315aefdeb883229c8e495f2317aed46c4f585d5 Mon Sep 17 00:00:00 2001 From: Marble Ma Date: Fri, 28 Jan 2022 09:31:05 +0000 Subject: [PATCH 81/84] Fix create or replace table to insert occur no table error --- src/Storages/StorageDistributed.cpp | 10 ++++++++- src/Storages/StorageDistributed.h | 2 ++ src/Storages/StorageMergeTree.cpp | 23 +++++++++++++++++++++ src/Storages/StorageMergeTree.h | 4 ++++ src/Storages/StorageReplicatedMergeTree.cpp | 18 ++++++++++++++++ src/Storages/StorageReplicatedMergeTree.h | 2 ++ 6 files changed, 58 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 65ecf0c2496f..f25a25c0b2cd 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1425,6 +1425,14 @@ void registerStorageEmbeddedDistributed(StorageFactory & factory) distributed_settings.monitor_max_sleep_time_ms = Poco::Timespan(context->getSettingsRef().distributed_directory_monitor_max_sleep_time_ms); + String name_part = args.engine_name.substr(0, args.engine_name.size() - strlen("MergeTree")); + bool replicated = startsWith(name_part, "Replicated"); + String storage_policy; + if (replicated) + storage_policy = std::make_unique(args.getContext()->getReplicatedMergeTreeSettings()).get()->storage_policy; + else + storage_policy = std::make_unique(args.getContext()->getMergeTreeSettings()).get()->storage_policy; + return StorageDistributed::create( args.table_id, args.columns, @@ -1435,7 +1443,7 @@ void registerStorageEmbeddedDistributed(StorageFactory & factory) "store", context, nullptr, - "default", + storage_policy, args.relative_data_path, distributed_settings, args.attach); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 540ae413f19c..b5fc894f47ad 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -109,7 +109,9 @@ class StorageDistributed final : public shared_ptr_helper, p NamesAndTypesList getVirtuals() const override; /// Used by InterpreterInsertQuery + void setRemoteDatabaseName(const String & remote_database_) { remote_database = remote_database_; } std::string getRemoteDatabaseName() const { return remote_database; } + void setRemoteTableName(const String & remote_table_) { remote_table = remote_table_; } std::string getRemoteTableName() const { return remote_table; } /// Returns empty string if tables is used by TableFunctionRemote std::string getClusterName() const { return cluster_name; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 0d1c0684624d..e0640a8222bb 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -1506,6 +1507,28 @@ void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type) background_moves_assignee.trigger(); } +void StorageMergeTree::rename(const String & new_path_to_table_data, const StorageID & new_table_id) +{ + renameInMemory(new_table_id); + MergeTreeData::rename(new_path_to_table_data, new_table_id); +} + +void StorageMergeTree::renameInMemory(const StorageID & new_table_id) +{ + if (embedded_distributed) + { + auto embedded_distributed_table = std::dynamic_pointer_cast(embedded_distributed); + if (!embedded_distributed_table) + throw Exception( + "Table " + getStorageID().getNameForLogs() + " embedded_distributed only for StorageDistributed table engine.", + ErrorCodes::NOT_IMPLEMENTED); + embedded_distributed_table->setRemoteDatabaseName(new_table_id.database_name); + embedded_distributed_table->setRemoteTableName(new_table_id.table_name); + embedded_distributed->renameInMemory(new_table_id); + } + IStorage::renameInMemory(new_table_id); +} + CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context) { CheckResults results; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 87f022a051b0..64fd86a89751 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -94,6 +94,10 @@ class StorageMergeTree final : public shared_ptr_helper, publi void onActionLockRemove(StorageActionBlockType action_type) override; + void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; + + void renameInMemory(const StorageID & new_table_id) override; + CheckResults checkData(const ASTPtr & query, ContextPtr context) override; RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f5e385773931..32be237a6e88 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include @@ -5041,6 +5042,7 @@ void StorageReplicatedMergeTree::checkTableCanBeRenamed() const void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { checkTableCanBeRenamed(); + renameInMemory(new_table_id); MergeTreeData::rename(new_path_to_table_data, new_table_id); /// Update table name in zookeeper @@ -5062,6 +5064,22 @@ void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, c /// TODO: You can update names of loggers. } +void StorageReplicatedMergeTree::renameInMemory(const StorageID & new_table_id) +{ + if (embedded_distributed) + { + auto embedded_distributed_table = std::dynamic_pointer_cast(embedded_distributed); + if (!embedded_distributed_table) + throw Exception( + "Table " + getStorageID().getNameForLogs() + " embedded_distributed only for StorageDistributed table engine.", + ErrorCodes::NOT_IMPLEMENTED); + embedded_distributed_table->setRemoteDatabaseName(new_table_id.database_name); + embedded_distributed_table->setRemoteTableName(new_table_id.table_name); + embedded_distributed->renameInMemory(new_table_id); + } + IStorage::renameInMemory(new_table_id); +} + bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) const { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index cd556fe9e5db..f2134c2bb262 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -146,6 +146,8 @@ class StorageReplicatedMergeTree final : public shared_ptr_helper Date: Fri, 28 Jan 2022 15:40:46 +0800 Subject: [PATCH 82/84] Fix distributed subquery max_query_size limitation inconsistency --- src/Interpreters/executeQuery.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ee61ac9198f3..08dc96fc1577 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -402,9 +402,10 @@ static std::tuple executeQueryImpl( ASTPtr ast; const char * query_end; - /// Don't limit the size of internal queries. - size_t max_query_size = 0; - if (!internal) max_query_size = settings.max_query_size; + size_t max_query_size = settings.max_query_size; + /// Don't limit the size of internal queries or distributed subquery. + if (internal || client_info.query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) + max_query_size = 0; String query_database; String query_table; From 3d0871d60523d5a3ba0c72ec0bff21f84b41fbf4 Mon Sep 17 00:00:00 2001 From: madianjun Date: Thu, 27 Jan 2022 12:45:37 +0800 Subject: [PATCH 83/84] Fix JOIN and WITH TOTALS --- src/Client/GRPCClient.cpp | 15 ++-- src/Client/GRPCClient.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 13 ++- .../QueryPlan/DistributedPlanner.cpp | 80 ++++++++++++------- src/Processors/QueryPlan/DistributedPlanner.h | 1 + .../QueryPlan/DistributedSourceStep.cpp | 2 +- .../QueryPlan/DistributedSourceStep.h | 2 +- .../QueryPlan/MergingAggregatedStep.cpp | 3 + src/Processors/Sources/DistributedSource.cpp | 12 +-- .../DistributedSourceExecutor.cpp | 34 +++++--- src/QueryPipeline/DistributedSourceExecutor.h | 6 +- src/Server/GRPCServer.cpp | 6 +- 12 files changed, 111 insertions(+), 65 deletions(-) diff --git a/src/Client/GRPCClient.cpp b/src/Client/GRPCClient.cpp index 8e61bb9ce6bf..e84a3bf35d0f 100644 --- a/src/Client/GRPCClient.cpp +++ b/src/Client/GRPCClient.cpp @@ -28,10 +28,10 @@ namespace ErrorCodes extern const int GRPC_CANCEL_ERROR; } -GRPCClient::GRPCClient(const String & addr_) +GRPCClient::GRPCClient(const String & addr_, const String & description_) { addr = addr_; - log = &Poco::Logger::get("GRPCClient(" + addr + ")"); + log = &Poco::Logger::get("GRPCClient(" + description_ + ")"); } GRPCResult GRPCClient::executePlanFragment(const GRPCQueryInfo & query_info) @@ -72,7 +72,6 @@ GRPCClient::MessageType GRPCClient::read(Block & block) { assert(inner_context); - LOG_DEBUG(log, "Start reading result from {}.", addr); GRPCResult result; if (inner_context->reader->Read(&result)) { @@ -80,8 +79,7 @@ GRPCClient::MessageType GRPCClient::read(Block & block) { LOG_ERROR( log, - "Read from {} failed, exception.code: {}, exception.text: {}.", - addr, + "Read failed, exception.code: {}, exception.text: {}.", result.exception().code(), result.exception().display_text()); throw Exception(result.exception().display_text(), result.exception().code(), true); @@ -93,7 +91,6 @@ GRPCClient::MessageType GRPCClient::read(Block & block) ReadBufferFromString b(result.totals()); NativeReader reader(b, 0); block = reader.read(); - LOG_DEBUG(log, "Read totals from {} success, result size: {}, block rows: {}.", addr, result.output().size(), block.rows()); return MessageType::Totals; } @@ -102,7 +99,6 @@ GRPCClient::MessageType GRPCClient::read(Block & block) ReadBufferFromString b(result.extremes()); NativeReader reader(b, 0); block = reader.read(); - LOG_DEBUG(log, "Read extremes from {} success, result size: {}, block rows: {}.", addr, result.output().size(), block.rows()); return MessageType::Extremes; } @@ -111,7 +107,6 @@ GRPCClient::MessageType GRPCClient::read(Block & block) ReadBufferFromString b(result.output()); NativeReader reader(b, 0); block = reader.read(); - LOG_DEBUG(log, "Read data from {} success, result size: {}, block rows: {}.", addr, result.output().size(), block.rows()); } return MessageType::Data; } @@ -133,7 +128,7 @@ void GRPCClient::cancel() if (status.ok()) { if (result.cancelled()) - LOG_INFO(log, "Cancel success from {}, plan fragment id: {}", addr, plan_fragment_id); + LOG_DEBUG(log, "Cancel success, plan fragment id: {}", plan_fragment_id); else { throw Exception("Cancel failed from " + addr + ", plan fragment id: " + plan_fragment_id + ", code: " + toString(result.exception().code()) + ", " + result.exception().display_text(), ErrorCodes::GRPC_CANCEL_ERROR, true); @@ -142,7 +137,7 @@ void GRPCClient::cancel() else { LOG_ERROR( - log, "Cancel failed from {}, code: {}, plan fragment id: {}.", addr, status.error_code(), plan_fragment_id); + log, "Cancel failed, code: {}, plan fragment id: {}.", status.error_code(), plan_fragment_id); throw Exception(status.error_message() + ", " + result.exception().display_text(), ErrorCodes::GRPC_CANCEL_ERROR, true); } } diff --git a/src/Client/GRPCClient.h b/src/Client/GRPCClient.h index 17ef401cbd54..55471dc037b8 100644 --- a/src/Client/GRPCClient.h +++ b/src/Client/GRPCClient.h @@ -34,7 +34,7 @@ class GRPCClient MAX = Extremes, }; public: - GRPCClient(const String & addr_); + GRPCClient(const String & addr_, const String & description_); ~GRPCClient() = default; /// Send params of plan fragment to remote, and execute it. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index b002077d8c6a..6bba2f0b363f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1857,8 +1857,17 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc const auto & ast = query_info.query->as(); InterpreterParamsPtr interpreter_params = std::make_shared(context, ast); query_plan.addStep(std::move(prepared_count), std::move(interpreter_params)); - /// Build query plan for the first stage both on compute and store nodes, therefore we can get the same original query plan. - /// If trivial count is optimized, skip executeWhere. + if (context->isInitialQuery() && context->getRunningMode() == Context::RunningMode::STORE) + { + /// If initial query is running on store worker, skip first stage. + from_stage = QueryProcessingStage::WithMergeableState; + analysis_result.first_stage = false; + } + else + { + /// Build query plan for the first stage both on compute and store nodes, therefore we can get the same original query plan. + /// If trivial count is optimized, skip executeWhere. + } analysis_result.optimize_trivial_count = true; return; } diff --git a/src/Processors/QueryPlan/DistributedPlanner.cpp b/src/Processors/QueryPlan/DistributedPlanner.cpp index 8ce2f2822afd..94ca575c6785 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.cpp +++ b/src/Processors/QueryPlan/DistributedPlanner.cpp @@ -154,6 +154,33 @@ void DistributedPlanner::checkShuffle( } } +void DistributedPlanner::transferInterpreterParams(QueryPlan::Node *current_node, QueryPlan::Node *last_node) +{ + if (last_node->interpreter_params) + { + if (!current_node->interpreter_params) + { + current_node->interpreter_params = last_node->interpreter_params; + LOG_DEBUG( + log, + "Set context({} <= {}) to {}", + current_node->step->getName(), + last_node->step->getName(), + static_cast(current_node->interpreter_params->context.get())); + } + else + { + current_node->interpreter_params->group_by_with_totals |= last_node->interpreter_params->group_by_with_totals; + LOG_DEBUG( + log, + "Merge group_by_with_totals({} <= {}) to {}", + current_node->step->getName(), + last_node->step->getName(), + current_node->interpreter_params->group_by_with_totals); + } + } +} + void DistributedPlanner::buildStages() { LOG_DEBUG(log, "===> Build stages."); @@ -244,16 +271,7 @@ void DistributedPlanner::buildStages() } /// Transfer interpreter params bottom-up. - if (!frame.node->interpreter_params && last_node->interpreter_params) - { - frame.node->interpreter_params = last_node->interpreter_params; - LOG_DEBUG( - log, - "Set context({} <= {}) to {}", - frame.node->step->getName(), - last_node->step->getName(), - static_cast(frame.node->interpreter_params->context.get())); - } + transferInterpreterParams(frame.node, last_node); ++frame.visited_children; one_child_is_visited = false; @@ -291,7 +309,9 @@ void DistributedPlanner::buildStages() /// Append result stage for converging data. ++stage_id; + /// Create a virtual node, used in iterating stages. + /// It will be recreated in sending stages if needed. parent_stages.push(last_stage); auto step = std::make_unique(stage_id, last_stage->id, context); query_plan.nodes.emplace_back(QueryPlan::Node{ @@ -648,7 +668,7 @@ void DistributedPlanner::scheduleStages(PlanResult & plan_result) parent_stage->id, *result_stage->workers.front(), false, - false, + parent_stage_node->interpreter_params->group_by_with_totals, context); query_plan.addStep(std::move(distributed_source_step)); plan_result.distributed_source_nodes.emplace_back(query_plan.root); @@ -850,7 +870,7 @@ void DistributedPlanner::scheduleStages(PlanResult & plan_result) query_info.set_has_view_source(stage.maybe_has_view_source); query_info.set_has_input_function(stage.has_input_function); - GRPCClient cli(*worker); + GRPCClient cli(*worker, "initiator"); auto result = cli.executePlanFragment(query_info); LOG_DEBUG( log, @@ -923,16 +943,7 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) if (one_child_is_visited) { /// Transfer interpreter params bottom-up. - if (!frame.node->interpreter_params && last_node->interpreter_params) - { - frame.node->interpreter_params = last_node->interpreter_params; - LOG_DEBUG( - log, - "Set context({} <= {}) to {}", - frame.node->step->getName(), - last_node->step->getName(), - static_cast(frame.node->interpreter_params->context.get())); - } + transferInterpreterParams(frame.node, last_node); CheckShuffleResult result; checkShuffle(frame.node, last_node, result, stage_seq, leaf_nodes); @@ -979,12 +990,25 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) ? result.grandchild_step->getOutputStream().header : aggregating_step->getOutputStream().header); + bool add_totals = false; + if (last_node->interpreter_params) + { + add_totals = last_node->interpreter_params->group_by_with_totals; + LOG_DEBUG(log, "======== group_by_with_totals {}", last_node->interpreter_params->group_by_with_totals); + } /// Create DistributedSourceStep. assert(header); const auto & sources = it->second; auto distributed_source_step = std::make_unique( - header, sources, query_distributed_plan_info.initial_query_id, my_stage_id, stage_id, my_replica, - add_agg_info, false, context); + header, + sources, + query_distributed_plan_info.initial_query_id, + my_stage_id, + stage_id, + my_replica, + add_agg_info, + add_totals, + context); QueryPlan::Node * new_node = nullptr; addStep(std::move(distributed_source_step), "", new_node); plan_result.distributed_source_nodes.emplace_back(new_node); /// For debug @@ -994,10 +1018,10 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) /// If parent stage has aggregate, add MergingAggregatedStep. if (result.child_aggregating_step) { - assert(frame.node->interpreter_params); - bool aggregate_final = !frame.node->interpreter_params->group_by_with_totals - && !frame.node->interpreter_params->group_by_with_rollup - && !frame.node->interpreter_params->group_by_with_cube; + assert(last_node->interpreter_params); + bool aggregate_final = !last_node->interpreter_params->group_by_with_totals + && !last_node->interpreter_params->group_by_with_rollup + && !last_node->interpreter_params->group_by_with_cube; LOG_DEBUG(log, "MergingAggregatedStep final: {}", aggregate_final); auto transform_params = std::make_shared(aggregating_step->getParams(), aggregate_final); diff --git a/src/Processors/QueryPlan/DistributedPlanner.h b/src/Processors/QueryPlan/DistributedPlanner.h index 3e8d4cd97845..cce94751a39b 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.h +++ b/src/Processors/QueryPlan/DistributedPlanner.h @@ -104,6 +104,7 @@ class DistributedPlanner { String debugRemotePlanFragment(const String & query, const String & receiver, const String & query_id, const Stage * stage); bool isSinglePointDataSource(const String & name); + void transferInterpreterParams(QueryPlan::Node *current_node, QueryPlan::Node *last_node); void buildStages(); void debugStages(); diff --git a/src/Processors/QueryPlan/DistributedSourceStep.cpp b/src/Processors/QueryPlan/DistributedSourceStep.cpp index 45cb0463ee74..3e7308aad60b 100644 --- a/src/Processors/QueryPlan/DistributedSourceStep.cpp +++ b/src/Processors/QueryPlan/DistributedSourceStep.cpp @@ -61,7 +61,7 @@ DistributedSourceStep::DistributedSourceStep( void DistributedSourceStep::addPipe(Pipes & pipes, const std::shared_ptr & source) { - auto distributed_source_executor = std::make_shared(header, source, query_id, node_id, parent_stage_id); + auto distributed_source_executor = std::make_shared(header, source, query_id, node_id, stage_id, parent_stage_id); pipes.emplace_back(createDistributedSourcePipe( distributed_source_executor, add_aggregation_info, add_totals, context->getSettingsRef().extremes, false)); pipes.back().addInterpreterContext(context); diff --git a/src/Processors/QueryPlan/DistributedSourceStep.h b/src/Processors/QueryPlan/DistributedSourceStep.h index 2fa3743ce589..fc9c69587933 100644 --- a/src/Processors/QueryPlan/DistributedSourceStep.h +++ b/src/Processors/QueryPlan/DistributedSourceStep.h @@ -31,7 +31,7 @@ class DistributedSourceStep final : public ISourceStep int parent_stage_id_, const String & node_id_, bool add_aggregation_info_, - bool add_totals_, /// Never use, should always be false. + bool add_totals_, ContextPtr context_); String getName() const override { return "DistributedSourceStep(" + toString(stage_id) + " <= " + toString(parent_stage_id) + ")"; } diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index 8dfb9f9c9239..19012e4ba15d 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -42,6 +42,9 @@ MergingAggregatedStep::MergingAggregatedStep( void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { + /// Forget about current totals and extremes. They will be calculated again after aggregation if needed. + pipeline.dropTotalsAndExtremes(); + if (!memory_efficient_aggregation) { /// We union several sources into one, paralleling the work. diff --git a/src/Processors/Sources/DistributedSource.cpp b/src/Processors/Sources/DistributedSource.cpp index 46b584e7b048..af91f78a72a5 100644 --- a/src/Processors/Sources/DistributedSource.cpp +++ b/src/Processors/Sources/DistributedSource.cpp @@ -134,15 +134,15 @@ Chunk DistributedExtremesSource::generate() } Pipe createDistributedSourcePipe( - DistributedSourceExecutorPtr executor, bool add_aggregation_info, bool /*add_totals*/, bool /*add_extremes*/, bool async_read) + DistributedSourceExecutorPtr executor, bool add_aggregation_info, bool add_totals, bool add_extremes, bool async_read) { Pipe pipe(std::make_shared(executor, add_aggregation_info, async_read)); -// if (add_totals) -// pipe.addTotalsSource(std::make_shared(executor)); -// -// if (add_extremes) -// pipe.addExtremesSource(std::make_shared(executor)); + if (add_totals) + pipe.addTotalsSource(std::make_shared(executor)); + + if (add_extremes) + pipe.addExtremesSource(std::make_shared(executor)); return pipe; } diff --git a/src/QueryPipeline/DistributedSourceExecutor.cpp b/src/QueryPipeline/DistributedSourceExecutor.cpp index 7d392fd09f9a..2ae9a69224f5 100644 --- a/src/QueryPipeline/DistributedSourceExecutor.cpp +++ b/src/QueryPipeline/DistributedSourceExecutor.cpp @@ -4,22 +4,30 @@ namespace DB { DistributedSourceExecutor::DistributedSourceExecutor( - Block header_, + const Block & header_, const std::shared_ptr & source_, const String & query_id_, const String & node_id_, - int stage_id_) - : header(std::move(header_)) + int stage_id_, + int parent_stage_id_) + : header(header_) + , totals(header_) + , extremes(header_) , source(source_) , query_id(query_id_) , node_id(node_id_) , stage_id(stage_id_) - , client(*source_) - , log(&Poco::Logger::get("DistributedSourceExecutor(" + query_id + "/" + toString(stage_id) + "/" + *source + ")")) + , parent_stage_id(parent_stage_id_) + , client(*source_, query_id + "/" + toString(stage_id) + "/" + node_id + "<=" + toString(parent_stage_id) + "/" + *source) + , log(&Poco::Logger::get( + "DistributedSourceExecutor(" + query_id + "/" + toString(stage_id) + "/" + node_id + "<=" + toString(parent_stage_id) + "/" + + *source + ")")) { + /// Stage A(host is a) read data from Stage B(host is b), The ticket sent from a to b is : + /// ticket{stage_id = B, node_id = a} GRPCTicket ticket; ticket.set_initial_query_id(query_id); - ticket.set_stage_id(stage_id); + ticket.set_stage_id(parent_stage_id); ticket.set_node_id(node_id); client.prepareRead(ticket); } @@ -42,14 +50,16 @@ Block DistributedSourceExecutor::read() case GRPCClient::MessageType::Data: if (!block) finished = true; - LOG_DEBUG(log, "Read block, rows: {}, columns: {}.", block.rows(), block.columns()); + LOG_DEBUG(log, "Read data block, rows: {}, columns: {}.", block.rows(), block.columns()); return block; case GRPCClient::MessageType::Totals: + LOG_DEBUG(log, "Read totals block, rows: {}, columns: {}.", block.rows(), block.columns()); totals = block; break; case GRPCClient::MessageType::Extremes: + LOG_DEBUG(log, "Read extremes block, rows: {}, columns: {}.", block.rows(), block.columns()); extremes = block; break; @@ -70,8 +80,8 @@ void DistributedSourceExecutor::finish() if (!isQueryPending() || hasThrownException()) return; - LOG_DEBUG(log, "Finish reading from {}.", *source); - tryCancel("Cancelling query because enough data has been read"); + LOG_DEBUG(log, "Finish reading."); + tryCancel("Enough data has been read"); finished = true; } @@ -81,9 +91,9 @@ void DistributedSourceExecutor::cancel() if (!isQueryPending() || hasThrownException()) return; - LOG_DEBUG(log, "Cancel reading from {}.", *source); + LOG_DEBUG(log, "Cancel reading."); client.cancel(); - tryCancel("Cancelling query"); + tryCancel("Query is cancelled"); } void DistributedSourceExecutor::tryCancel(const char * reason) @@ -96,7 +106,7 @@ void DistributedSourceExecutor::tryCancel(const char * reason) was_cancelled = true; - LOG_TRACE(log, "({}) {}", *source, reason); + LOG_TRACE(log, "Reason: {}.", reason); } bool DistributedSourceExecutor::isQueryPending() const diff --git a/src/QueryPipeline/DistributedSourceExecutor.h b/src/QueryPipeline/DistributedSourceExecutor.h index 630bcf8077d0..df441ee2b651 100644 --- a/src/QueryPipeline/DistributedSourceExecutor.h +++ b/src/QueryPipeline/DistributedSourceExecutor.h @@ -10,11 +10,12 @@ class DistributedSourceExecutor public: /// Takes already set connection. DistributedSourceExecutor( - Block header_, + const Block & header_, const std::shared_ptr & source_, const String & query_id_, const String & node_id_, - int stage_id_); + int stage_id_, + int parent_stage_id_); ~DistributedSourceExecutor(); @@ -36,6 +37,7 @@ class DistributedSourceExecutor String query_id; String node_id; int stage_id; + int parent_stage_id; GRPCClient client; Poco::Logger * log = nullptr; std::atomic finished{false}; diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 124f79069680..7ca661d651c6 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1727,7 +1727,7 @@ namespace query_info_wrapper->notifyHeader(io.pipeline.getHeader()); /// Pull block from pipeline. - auto executor = std::make_shared(io.pipeline); + auto executor = std::make_shared(io.pipeline); auto check_for_cancel = [this, &executor] { if (query_info_wrapper->cancel) { @@ -1743,7 +1743,7 @@ namespace Block block; while (check_for_cancel()) { - if (!executor->pull(block, interactive_delay / 1000)) + if (!executor->pull(block)) break; if (!check_for_cancel()) @@ -1770,6 +1770,7 @@ namespace query_info_wrapper->totals = executor->getTotalsBlock(); query_info_wrapper->extremes = executor->getExtremesBlock(); query_info_wrapper->profile_info = executor->getProfileInfo(); + LOG_DEBUG(log, "{} produce totals {}, extremes {}.", query_info_key, query_info_wrapper->totals.rows(), query_info_wrapper->extremes.rows()); } query_info_wrapper->notifyFinish(); } @@ -1837,6 +1838,7 @@ namespace if (!query_info_wrapper->cancel) { + LOG_DEBUG(log, "{}|{} consume totals {}, extremes {}.", query_info_key, ticket.node_id(), query_info_wrapper->totals.rows(), query_info_wrapper->extremes.rows()); addTotalsToResult(query_info_wrapper->totals); addExtremesToResult(query_info_wrapper->extremes); addProfileInfoToResult(query_info_wrapper->profile_info); From 9af4f3440ea3fbfcd8c064489bff882949fc98ea Mon Sep 17 00:00:00 2001 From: madianjun Date: Fri, 28 Jan 2022 12:23:42 +0800 Subject: [PATCH 84/84] Change stage building in right join and full join --- .../QueryPlan/DistributedPlanner.cpp | 135 +++++++++++++----- src/Processors/QueryPlan/QueryPlan.h | 9 ++ 2 files changed, 109 insertions(+), 35 deletions(-) diff --git a/src/Processors/QueryPlan/DistributedPlanner.cpp b/src/Processors/QueryPlan/DistributedPlanner.cpp index 94ca575c6785..8cf98f39c5b1 100644 --- a/src/Processors/QueryPlan/DistributedPlanner.cpp +++ b/src/Processors/QueryPlan/DistributedPlanner.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include #include #include #include @@ -60,8 +62,8 @@ void DistributedPlanner::checkShuffle( result.current_union_step = dynamic_cast(current_node->step.get()); if (result.current_union_step) { - LOG_DEBUG(log, "[{}]Check shuffle: child node is UnionStep", getStageSeqName(stage_seq)); result.is_shuffle = true; + LOG_DEBUG(log, "[{}]Check shuffle: {}, current node is UnionStep", getStageSeqName(stage_seq), result.is_shuffle); stage_seq = StageSeq::STAGE1; return; } @@ -69,22 +71,65 @@ void DistributedPlanner::checkShuffle( result.current_join_step = dynamic_cast(current_node->step.get()); if (result.current_join_step) { - LOG_DEBUG(log, "[{}]Check shuffle: current node is JoinStep(0x{})", getStageSeqName(stage_seq), static_cast(result.current_join_step)); assert(current_node->children.size() == 2); - /// Only broadcast right side. - if (child_node == current_node->children[1]) + bool maybe_need_shuffle = false; + const auto join_kind = result.current_join_step->getJoin()->getTableJoin().kind(); + if (isFull((join_kind))) { - if (child_node->num_parent_stages == 0 && child_node->num_leaf_nodes_in_stage == 1 - && !leaf_nodes.empty() && isSinglePointDataSource(leaf_nodes.top()->step->getStepDescription())) + result.is_shuffle = true; + LOG_DEBUG(log, "[{}]Check shuffle: {}, current node is JoinStep(FULL JOIN)", getStageSeqName(stage_seq), result.is_shuffle); + return; + } + else if (isRight(join_kind)) + { + if (child_node == current_node->children[1]) + { + result.is_shuffle = false; + LOG_DEBUG(log, "[{}]Check shuffle: {}, current node is JoinStep(RIGHT JOIN && right side)", getStageSeqName(stage_seq), result.is_shuffle); + } + else + maybe_need_shuffle = true; + } + else if (isLeft(join_kind)) + { + if (child_node == current_node->children[0]) + { + result.is_shuffle = false; + LOG_DEBUG(log, "[{}]Check shuffle: {}, current node is JoinStep(LEFT JOIN && left side)", getStageSeqName(stage_seq), result.is_shuffle); + } + else + maybe_need_shuffle = true; + } + else if (child_node == current_node->children[1]) + { + maybe_need_shuffle = true; + } + + if (maybe_need_shuffle) + { + /// Broadcast one side: + /// 1. LEFT JOIN && child is right: broadcast right side. + /// 2. RIGHT JOIN && child is left: broadcast left side. + /// 3. Other cases: broadcast right side. + if (child_node->num_parent_stages == 0 && child_node->num_leaf_nodes_in_stage == 1 && !leaf_nodes.empty() + && isSinglePointDataSource(leaf_nodes.top()->step->getStepDescription())) { /// 1. Storage name is SystemNumbers: select ... from t1, numbers(10) n /// 2. Storage name is Memory: select ... from t1 join (select number from numbers(1)) /// 3. Storage name is Memory: select ... from t1 join t2 - LOG_DEBUG(log, "No need shuffle: right child's storage is {}", leaf_nodes.top()->step->getStepDescription()); result.is_shuffle = false; + LOG_DEBUG( + log, + "[{}]Check shuffle: {}, current node is JoinStep, child's storage is {}", + getStageSeqName(stage_seq), + result.is_shuffle, + leaf_nodes.top()->step->getStepDescription()); } else + { result.is_shuffle = true; + LOG_DEBUG(log, "[{}]Check shuffle: {}, current node is JoinStep", getStageSeqName(stage_seq), result.is_shuffle); + } } stage_seq = StageSeq::STAGE1; return; @@ -95,9 +140,9 @@ void DistributedPlanner::checkShuffle( { /// From : AggregatingStep => /// To : AggregatingStep(partial) [=> MergingAggregatedStep(final)] => - LOG_DEBUG(log, "[{}]Check shuffle: child node is AggregatingStep", getStageSeqName(stage_seq)); result.grandchild_step = child_node->children.front()->step.get(); result.is_shuffle = true; + LOG_DEBUG(log, "[{}]Check shuffle: {}, child node is AggregatingStep", getStageSeqName(stage_seq), result.is_shuffle); stage_seq = StageSeq::STAGE2; return; } @@ -107,14 +152,15 @@ void DistributedPlanner::checkShuffle( { /// From : SortingStep => Not (LimitStep) => /// To : SortingStep(partial) [=> SortingStep(final)] => Not (LimitStep) => - LOG_DEBUG(log, "[{}]Check shuffle: child node is SortingStep", getStageSeqName(stage_seq)); if (stage_seq == StageSeq::STAGE2) { result.is_shuffle = false; + LOG_DEBUG(log, "[{}]Check shuffle: {}, child node is SortingStep", getStageSeqName(stage_seq), result.is_shuffle); } else if (!(result.current_limit_step = dynamic_cast(current_node->step.get()))) { result.is_shuffle = true; + LOG_DEBUG(log, "[{}]Check shuffle: {}, child node is SortingStep", getStageSeqName(stage_seq), result.is_shuffle); stage_seq = StageSeq::STAGE2; } return; @@ -122,12 +168,12 @@ void DistributedPlanner::checkShuffle( if ((result.child_distinct_step = dynamic_cast(child_node->step.get()))) { - LOG_DEBUG(log, "[{}]Check shuffle: child node is DistinctStep", getStageSeqName(stage_seq)); result.current_distinct_step = dynamic_cast(current_node->step.get()); if (result.current_distinct_step) { /// DistinctStep(partial) => (shuffle) => DistinctStep(final) => result.is_shuffle = true; + LOG_DEBUG(log, "[{}]Check shuffle: {}, both of child and current nodes are DistinctStep", getStageSeqName(stage_seq), result.is_shuffle); stage_seq = StageSeq::STAGE2; } return; @@ -160,23 +206,34 @@ void DistributedPlanner::transferInterpreterParams(QueryPlan::Node *current_node { if (!current_node->interpreter_params) { - current_node->interpreter_params = last_node->interpreter_params; + if (current_node->children.size() == 1) + current_node->interpreter_params = last_node->interpreter_params; + else + current_node->interpreter_params = std::make_shared(*last_node->interpreter_params); LOG_DEBUG( log, - "Set context({} <= {}) to {}", + "Set context({}({}) <= {}({})) to {}", current_node->step->getName(), + current_node->interpreter_params->group_by_with_totals, last_node->step->getName(), + last_node->interpreter_params->group_by_with_totals, static_cast(current_node->interpreter_params->context.get())); } - else + else if (current_node->children.size() > 1) { - current_node->interpreter_params->group_by_with_totals |= last_node->interpreter_params->group_by_with_totals; + /// For Join and Union, not for ITransformingStep. + bool merged_with_totals = current_node->interpreter_params->group_by_with_totals | last_node->interpreter_params->group_by_with_totals; LOG_DEBUG( log, - "Merge group_by_with_totals({} <= {}) to {}", + "Merge group_by_with_totals({}({}/{}) <= {}({}/{})) to {}", current_node->step->getName(), + static_cast(current_node->interpreter_params.get()), + current_node->interpreter_params->group_by_with_totals, last_node->step->getName(), - current_node->interpreter_params->group_by_with_totals); + static_cast(last_node->interpreter_params.get()), + last_node->interpreter_params->group_by_with_totals, + merged_with_totals); + current_node->interpreter_params->group_by_with_totals = merged_with_totals; } } } @@ -185,7 +242,12 @@ void DistributedPlanner::buildStages() { LOG_DEBUG(log, "===> Build stages."); - auto createStage = [this](int id, std::stack & parent_stages, QueryPlan::Node * root_node, std::stack & leaf_nodes) { + StageSeq stage_seq = StageSeq::STAGE1; + auto createStage = [this, &stage_seq]( + int id, + std::stack & parent_stages, + QueryPlan::Node * root_node, + std::stack & leaf_nodes) { stages.emplace_back(Stage{.id = id, .root_node = root_node}); Stage * new_stage = &stages.back(); @@ -212,7 +274,7 @@ void DistributedPlanner::buildStages() leaf_nodes.pop(); } } - LOG_DEBUG(log, "Create stage: id: {}, has {} parent stages and {} leaf nodes.", id, new_stage->parents.size(), new_stage->leaf_nodes.size()); + LOG_DEBUG(log, "[{}]Create stage: id: {}({} parent stages, {} leaf nodes).",getStageSeqName(stage_seq), id, new_stage->parents.size(), new_stage->leaf_nodes.size()); return new_stage; }; @@ -234,7 +296,6 @@ void DistributedPlanner::buildStages() Stage * last_stage = nullptr; std::stack parent_stages; std::stack leaf_nodes; - StageSeq stage_seq = StageSeq::STAGE1; while (!stack.empty()) { @@ -318,13 +379,15 @@ void DistributedPlanner::buildStages() .step = std::move(step), .children = {last_node}, .num_parent_stages = 1, - .interpreter_params = query_plan.root->interpreter_params}); + .interpreter_params = last_node->interpreter_params}); query_plan.root = &query_plan.nodes.back(); LOG_DEBUG( log, - "Set context({} <= {}) to {}", + "Set context({}({}) <= {}({})) to {}", query_plan.root->step->getName(), + query_plan.root->interpreter_params->group_by_with_totals, last_node->step->getName(), + last_node->interpreter_params->group_by_with_totals, static_cast(query_plan.root->interpreter_params->context.get())); /// Maintain leaf nodes. @@ -961,18 +1024,26 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) assert(last_node == frame.node->children[frame.visited_children]); /// Add steps between current node and child node. - auto addStep = [this, &stage_id, &frame](QueryPlanStepPtr step, const String & description, QueryPlan::Node * & node) + auto addStep = [this, &stage_id, &last_node, &stage_seq](QueryPlanStepPtr step, const String & description, QueryPlan::Node * & node) { - LOG_DEBUG(log, "Add step: {}, parent stage: {}", step->getName(), stage_id); + LOG_DEBUG(log, "[{}]Add step: {}, parent stage id: {}", getStageSeqName(stage_seq), step->getName(), stage_id); step->setStepDescription(description); if (!node) - query_plan.nodes.emplace_back(QueryPlan::Node{.step = std::move(step), .interpreter_params = frame.node->interpreter_params}); + query_plan.nodes.emplace_back(QueryPlan::Node{.step = std::move(step), .interpreter_params = last_node->interpreter_params}); else { - query_plan.nodes.emplace_back(QueryPlan::Node{.step = std::move(step), .children = {node}, .interpreter_params = frame.node->interpreter_params}); + query_plan.nodes.emplace_back(QueryPlan::Node{.step = std::move(step), .children = {node}, .interpreter_params = last_node->interpreter_params}); node->parent = &query_plan.nodes.back(); } node = &query_plan.nodes.back(); + LOG_DEBUG( + log, + "Set context({}({}) <= {}({})) to {}", + node->step->getName(), + node->interpreter_params->group_by_with_totals, + last_node->step->getName(), + last_node->interpreter_params->group_by_with_totals, + static_cast(node->interpreter_params->context.get())); }; bool add_agg_info = false; @@ -990,12 +1061,6 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) ? result.grandchild_step->getOutputStream().header : aggregating_step->getOutputStream().header); - bool add_totals = false; - if (last_node->interpreter_params) - { - add_totals = last_node->interpreter_params->group_by_with_totals; - LOG_DEBUG(log, "======== group_by_with_totals {}", last_node->interpreter_params->group_by_with_totals); - } /// Create DistributedSourceStep. assert(header); const auto & sources = it->second; @@ -1007,7 +1072,7 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) stage_id, my_replica, add_agg_info, - add_totals, + last_node->interpreter_params->group_by_with_totals, context); QueryPlan::Node * new_node = nullptr; addStep(std::move(distributed_source_step), "", new_node); @@ -1067,9 +1132,9 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) } else if (stage_id == my_stage_id) { - auto replaceStep = [this, &stage_id](QueryPlanStepPtr step, QueryPlan::Node * & node) + auto replaceStep = [this, &stage_id, &stage_seq](QueryPlanStepPtr step, QueryPlan::Node * & node) { - LOG_DEBUG(log, "Replace step: {}, stage: {}", step->getName(), stage_id); + LOG_DEBUG(log, "[{}]Replace step: {}, stage: {}", getStageSeqName(stage_seq), step->getName(), stage_id); node->step = std::move(step); }; @@ -1092,7 +1157,7 @@ void DistributedPlanner::buildPlanFragment(PlanResult & plan_result) /// If optimize trivial count, remove AggregatingStep. else { - LOG_DEBUG(log, "Remove step: {}, stage: {}", result.child_aggregating_step->getName(), stage_id); + LOG_DEBUG(log, "[{}]Remove step: {}, stage: {}", getStageSeqName(stage_seq), result.child_aggregating_step->getName(), stage_id); last_node = last_node->children[0]; } } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 03d17171ca4c..4f190f99627e 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -46,6 +46,15 @@ struct InterpreterParams group_by_with_rollup = query_ast_.group_by_with_rollup; group_by_with_cube = query_ast_.group_by_with_cube; } + + InterpreterParams(const InterpreterParams & interpreter_params) + : context(interpreter_params.context) + , group_by_with_totals(interpreter_params.group_by_with_totals) + , group_by_with_rollup(interpreter_params.group_by_with_rollup) + , group_by_with_cube(interpreter_params.group_by_with_cube) + { + } + ContextPtr context; bool group_by_with_totals; bool group_by_with_rollup;