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..f3c4952c232a 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 @@ -1546,6 +1562,9 @@ 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."); diff --git a/src/Client/GRPCClient.cpp b/src/Client/GRPCClient.cpp new file mode 100644 index 000000000000..e84a3bf35d0f --- /dev/null +++ b/src/Client/GRPCClient.cpp @@ -0,0 +1,144 @@ +#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; + extern const int GRPC_CANCEL_ERROR; +} + +GRPCClient::GRPCClient(const String & addr_, const String & description_) +{ + addr = addr_; + log = &Poco::Logger::get("GRPCClient(" + description_ + ")"); +} + +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; + grpc::Status status = stub->ExecutePlanFragment(&ctx, query_info, &result); + + if (status.ok()) + return result; + else + { + LOG_ERROR( + 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); + } +} + +void GRPCClient::prepareRead(const GRPCTicket & ticket_) +{ + ticket = ticket_; + 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); + inner_context = std::make_unique(ch, ctx, stub, reader); +} + +GRPCClient::MessageType GRPCClient::read(Block & block) +{ + assert(inner_context); + + GRPCResult result; + if (inner_context->reader->Read(&result)) + { + if (result.exception().code() != 0) + { + LOG_ERROR( + log, + "Read failed, exception.code: {}, exception.text: {}.", + result.exception().code(), + result.exception().display_text()); + throw Exception(result.exception().display_text(), result.exception().code(), true); + } + + /// 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(); + return MessageType::Totals; + } + + if (result.extremes().size() > 0) + { + ReadBufferFromString b(result.extremes()); + NativeReader reader(b, 0); + block = reader.read(); + return MessageType::Extremes; + } + + if (!result.output().empty()) + { + ReadBufferFromString b(result.output()); + NativeReader reader(b, 0); + block = reader.read(); + } + return MessageType::Data; + } + + 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.initial_query_id() + "/" + toString(ticket.stage_id()) + "/" + ticket.node_id(); + if (status.ok()) + { + if (result.cancelled()) + 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); + } + } + else + { + LOG_ERROR( + 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 new file mode 100644 index 000000000000..55471dc037b8 --- /dev/null +++ b/src/Client/GRPCClient.h @@ -0,0 +1,78 @@ +#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: + enum MessageType + { + Data = 1, + Totals = 2, + Extremes = 3, + MAX = Extremes, + }; +public: + GRPCClient(const String & addr_, const String & description_); + ~GRPCClient() = default; + + /// Send params of plan fragment to remote, and execute it. + GRPCResult executePlanFragment(const GRPCQueryInfo & query_info); + + /// 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. + MessageType read(Block & bock); + + /// Cancel plan fragment (ticket associated with the prepareRead) + void cancel(); + +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::unique_ptr inner_context; + GRPCTicket ticket; +}; +} +//#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..688fdc6d13a1 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -593,6 +593,9 @@ M(623, CAPN_PROTO_BAD_CAST) \ M(624, BAD_FILE_TYPE) \ 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/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/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/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..fa0e63f3c673 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 { @@ -359,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. @@ -382,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 { @@ -480,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/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..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>(); @@ -1133,4 +1222,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/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/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/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 09b2548f2f44..3472995bee4c 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,69 @@ 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()) + 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. + + 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); + auto pos = path.rfind("/"); + if (pos != std::string::npos) + { + if (pos == 0 && path.length() > 1) { + path = "/"; + } + else + { + path = path.substr(0, pos); + trimRight(path, '/'); + } + } + } } + }; } else @@ -788,9 +830,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 +1261,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; 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..4e44b54789ae 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::Children)) + 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/Core/Defines.h b/src/Core/Defines.h index 215bf6780d95..0b6b23aeb718 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -2,6 +2,16 @@ #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" +/// 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 #define DBMS_DEFAULT_SECURE_PORT 9440 #define DBMS_DEFAULT_CONNECT_TIMEOUT_SEC 10 @@ -65,6 +75,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/Core/Settings.h b/src/Core/Settings.h index e6f685a06502..346651a1b7e1 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) \ @@ -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) \ @@ -263,13 +263,14 @@ 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) \ 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. \ @@ -375,7 +376,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) \ @@ -482,10 +483,10 @@ 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) \ + 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) \ @@ -621,7 +622,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/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 3f253ed0924e..1ef6d004b2ae 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(); @@ -456,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"); @@ -819,6 +822,17 @@ 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, const std::function & add_ops) +{ + auto txn = query_context->getZooKeeperMetadataTransaction(); + if (txn && txn->isInitialQuery()) + { + if (add_ops) + add_ops(txn); + 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..f2e361641e6f 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, 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/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/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/ClientInfo.h b/src/Interpreters/ClientInfo.h index d42c34f07e2e..2446b8eaba01 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -57,11 +57,15 @@ 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{}; 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/Cluster.cpp b/src/Interpreters/Cluster.cpp index 30d0dd4cecee..544d50abcb0b 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,135 @@ bool Cluster::maybeCrossReplication() const return false; } +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() == '/') + 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); + + /// 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)); + + /// 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..f5fcf5cf3660 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -2,6 +2,8 @@ #include #include +#include +#include #include @@ -303,4 +305,38 @@ class Clusters mutable std::mutex mutex; }; +class ClustersWatcher +{ +public: + ClustersWatcher(const std::string & clusters_path_, ContextPtr context_); + + 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/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/Context.cpp b/src/Interpreters/Context.cpp index 7f31df9159ca..34edf27255ca 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 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. @@ -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 initial_contexts; /// Retrieve view source or input function in different plan fragments. ExternalLoaderXMLConfigRepository * external_models_config_repository = nullptr; scope_guard models_repository_guard; @@ -228,6 +230,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 +276,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 +376,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 @@ -891,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); @@ -915,7 +944,6 @@ Tables Context::getExternalTables() const return res; } - void Context::addExternalTable(const String & table_name, TemporaryTableHolder && temporary_table) { assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL); @@ -1053,6 +1081,28 @@ StoragePtr Context::getViewSource() const return view_source; } +void Context::addInitialContext(const String & plan_fragment_id, const ContextPtr & context) +{ + std::lock_guard lock(shared->initial_contexts_mutex); + shared->initial_contexts[plan_fragment_id] = context; +} + +ContextPtr Context::getInitialContext(const String & plan_fragment_id) const +{ + 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_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 { auto lock = getLock(); @@ -1158,6 +1208,12 @@ std::shared_ptr Context::getSettingsCons } +String Context::getSessionID() const +{ + auto lock = getLock(); + return session_id; +} + String Context::getCurrentDatabase() const { auto lock = getLock(); @@ -1185,6 +1241,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); @@ -1256,6 +1318,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) @@ -1795,6 +1885,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 +2742,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..39540a5a4143 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -15,6 +16,7 @@ #include #include #include +#include #include "config_core.h" @@ -54,6 +56,7 @@ class BackgroundSchedulePool; class MergeList; class ReplicatedFetchList; class Cluster; +class ClustersWatcher; class Compiler; class MarkCache; class MMappedFileCache; @@ -147,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 @@ -198,6 +203,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. @@ -294,10 +300,26 @@ class Context: public std::enable_shared_from_this public: + zkutil::DistributedRWLockPtr ddl_database_lock; + zkutil::DistributedRWLockPtr ddl_table_lock; + + struct QueryPlanFragmentInfo + { + 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. + 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. 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. + using SampleBlockCache = std::unordered_map; mutable SampleBlockCache sample_block_cache; @@ -462,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); @@ -500,22 +523,42 @@ 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_; } + void resetQueryPlanFragmentInfo() + { + if (query_plan_fragment_info) + query_plan_fragment_info.reset(); + } + StoragePtr executeTableFunction(const ASTPtr & table_expression); void addViewSource(const StoragePtr & storage); StoragePtr getViewSource() const; + void addInitialContext(const String & plan_fragment_id, const ContextPtr & context); + 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. void setCurrentDatabaseNameInGlobalContext(const String & name); void setCurrentQueryId(const String & query_id); + String generateQueryId() const; void killCurrentQuery(); @@ -741,6 +784,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 +873,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..85442e1ec1ba 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -350,9 +351,15 @@ 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(); + 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/DDLTask.h b/src/Interpreters/DDLTask.h index ee49274707a8..c07911cc1782 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); @@ -198,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 diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 205d7052d0c7..470f850f2167 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_ + DATABASE_LOCK_SUFFIX); + 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/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index d40a92972b20..6b04eff16194 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -411,7 +411,11 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global) { if (do_global) { - GlobalSubqueriesVisitor::Data subqueries_data(getContext(), subquery_depth, isRemoteStorage(), + /// 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/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index b6bb3c5fad51..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); }; @@ -219,6 +221,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/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 70f7c0c0359d..e42d7f1a0f9e 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(); @@ -229,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 55c007e2713b..09102eb3a707 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -1,8 +1,10 @@ #include +#include #include #include #include #include +#include namespace DB { @@ -12,6 +14,20 @@ void IInterpreterUnionOrSelectQuery::extendQueryLogElemImpl(QueryLogElement & el elem.query_kind = "Select"; } +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) + { + getContext()->resetQueryPlanFragmentInfo(); + 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() { @@ -19,8 +35,15 @@ QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline() buildQueryPlan(query_plan); + query_plan.checkInitialized(); + query_plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); + + DistributedPlanner planner(query_plan, context); + planner.buildDistributedPlan(); + + QueryPlanOptimizationSettings optimization_settings{.optimize_plan = false}; return std::move(*query_plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context))); + optimization_settings, BuildQueryPipelineSettings::fromContext(context))); } } diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index db9cc086e35b..5af88b48ef8a 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB @@ -13,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) @@ -25,8 +27,12 @@ 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 rewriteDistributedQuery(bool is_subquery, size_t tables_count = 0, bool need_log = false); + virtual void buildQueryPlan(QueryPlan & query_plan) = 0; QueryPipelineBuilder buildQueryPipeline(); @@ -40,8 +46,15 @@ class IInterpreterUnionOrSelectQuery : public IInterpreter void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const override; + 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/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 729a495987fe..381a55be7774 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include @@ -33,10 +34,11 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; extern const int NOT_IMPLEMENTED; extern const int TABLE_IS_READ_ONLY; + extern const int UNSUPPORTED_METHOD; } -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_) { } @@ -67,10 +69,13 @@ 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()) { + 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()); @@ -95,6 +100,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/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 6d38c55bd62c..a0095b57e05f 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -65,6 +65,7 @@ #include #include +#include namespace DB @@ -88,6 +89,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 +104,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) + { + 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 if (DatabaseCatalog::instance().isDatabaseExist(database_name)) @@ -132,19 +141,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,6 +246,15 @@ 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()); + if (!ptr) + throw Exception("The default database is not Replicated engine", ErrorCodes::LOGICAL_ERROR); + return ptr->tryEnqueueReplicatedDDL(query_ptr, getContext()); + } + DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", getContext()); if (create.uuid != UUIDHelpers::Nil) @@ -261,13 +282,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; @@ -827,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) { @@ -1104,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)); @@ -1176,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) @@ -1219,12 +1271,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/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 4fbad7e5471c..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 @@ -33,6 +34,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int INCORRECT_QUERY; extern const int TABLE_IS_READ_ONLY; + extern const int CANNOT_DROP_DATABASE; } @@ -80,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); @@ -270,9 +280,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); @@ -298,7 +314,11 @@ 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()->ddl_database_lock = zkutil::DistributedRWLock::tryLock(getContext()->getZooKeeper(), false, database_name); + } database = tryGetDatabase(database_name, query.if_exists); if (database) @@ -312,8 +332,17 @@ 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()); + if (!ptr) + throw Exception("The default database is not Replicated engine", ErrorCodes::LOGICAL_ERROR); + return ptr->tryEnqueueReplicatedDDL(query_ptr, getContext()); + } #if USE_MYSQL if (database->getEngineName() == "MaterializedMySQL") @@ -342,6 +371,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; @@ -351,6 +383,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) @@ -360,12 +394,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()); } 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 {}; } diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index e3d52487a52d..45cbb902db67 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -18,8 +19,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_) { } @@ -27,6 +28,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()); @@ -46,6 +51,21 @@ 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; + /// 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); @@ -56,6 +76,22 @@ BlockIO InterpreterRenameQuery::execute() table_guards[from]; table_guards[to]; + + /// Only add distributed lock when it's initial query + if (auto txn = getContext()->getZooKeeperMetadataTransaction() && rename.is_initial) + { + /// 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/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0050df1bf52d..6bba2f0b363f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -62,6 +63,7 @@ #include #include #include +#include #include #include @@ -347,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); @@ -399,7 +404,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; @@ -438,6 +444,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( std::move(subquery_for_sets), std::move(prepared_sets)); + /// For 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 +466,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 {}", it.first, static_cast(context.get())); + } } if (!options.only_analyze || options.modify_inplace) @@ -499,6 +512,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Calculate structure of the result. result_header = getSampleBlockImpl(); + + distributed_query_ptr = query_ptr->clone(); + rewriteDistributedQuery(false, joined_tables.tablesCount()); }; analyze(shouldMoveToPrewhere()); @@ -566,6 +582,27 @@ InterpreterSelectQuery::InterpreterSelectQuery( sanitizeBlock(result_header, true); } +void InterpreterSelectQuery::rewriteDistributedQuery(bool is_subquery, size_t tables_count, bool need_log) +{ + 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); + 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); +} + void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) { executeImpl(query_plan, std::move(input_pipe)); @@ -594,8 +631,15 @@ BlockIO InterpreterSelectQuery::execute() buildQueryPlan(query_plan); + query_plan.checkInitialized(); + query_plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); + + DistributedPlanner planner(query_plan, context); + planner.buildDistributedPlan(); + + QueryPlanOptimizationSettings optimization_settings{.optimize_plan = false}; res.pipeline = QueryPipelineBuilder::getPipeline(std::move(*query_plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)))); + optimization_settings, BuildQueryPipelineSettings::fromContext(context)))); return res; } @@ -1161,7 +1205,7 @@ 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) @@ -1474,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)); @@ -1511,8 +1561,9 @@ 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)); + 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)); if (query_info.projection) { @@ -1803,9 +1854,21 @@ 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)); - 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)); + 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; } } @@ -1886,6 +1949,8 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (query_analyzer->hasAggregation()) interpreter_subquery->ignoreWithTotals(); + + interpreter_subquery->rewriteDistributedQuery(true); } interpreter_subquery->buildQueryPlan(query_plan); @@ -2068,7 +2133,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/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 4298cbbb7943..85a4705fa006 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, bool need_log = false) override; + /// Execute a query. Get the stream of blocks to read. BlockIO execute() override; @@ -100,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 e7ea08e557d3..4357176f673e 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include @@ -34,6 +35,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(); @@ -165,7 +167,29 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( } options.ignore_limits |= all_nested_ignore_limits; options.ignore_quota |= all_nested_ignore_quota; +} + +void InterpreterSelectWithUnionQuery::rewriteDistributedQuery(bool is_subquery, size_t, bool need_log) +{ + 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 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); + 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); } Block InterpreterSelectWithUnionQuery::getCommonHeaderForUnion(const Blocks & headers) @@ -260,6 +284,7 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) if (num_plans == 1) { nested_interpreters.front()->buildQueryPlan(query_plan); + nested_interpreters.front()->rewriteDistributedQuery(false); } else { @@ -270,6 +295,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)) { @@ -328,8 +354,17 @@ BlockIO InterpreterSelectWithUnionQuery::execute() QueryPlan query_plan; buildQueryPlan(query_plan); + rewriteDistributedQuery(false, 0, true); + + query_plan.checkInitialized(); + query_plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); + + DistributedPlanner planner(query_plan, context); + planner.buildDistributedPlan(); + + QueryPlanOptimizationSettings optimization_settings{.optimize_plan = false}; auto pipeline_builder = query_plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), + optimization_settings, BuildQueryPipelineSettings::fromContext(context)); pipeline_builder->addInterpreterContext(context); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index 720632e7be5a..9a1af5596398 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, bool need_log = false) override; + /// Builds QueryPlan for current query. virtual void buildQueryPlan(QueryPlan & query_plan) override; @@ -41,6 +43,7 @@ class InterpreterSelectWithUnionQuery : public IInterpreterUnionOrSelectQuery private: std::vector> nested_interpreters; + Poco::Logger * log; static Block getCommonHeaderForUnion(const Blocks & headers); 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/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index e34d974fa805..aee14d523867 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(); } @@ -204,12 +206,31 @@ 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; + + 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 +926,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; }; 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/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) 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"); }; 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/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(); diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index c3b8cc5c6776..0595a19d28c0 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -37,8 +39,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 @@ -162,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); @@ -187,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; @@ -252,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")) @@ -295,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; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index f401f708ab14..08dc96fc1577 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -381,6 +381,7 @@ static std::tuple executeQueryImpl( 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, @@ -401,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; diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 65b2065b2adf..348f227cf72b 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -1,4 +1,5 @@ #include +#include #include @@ -163,6 +164,62 @@ 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)); + /// Sync databases with meta-service. + { + auto zookeeper = context->getZooKeeper(); + zookeeper->createAncestors(std::filesystem::path(DEFAULT_ZOOKEEPER_METADATA_PATH) / ""); + 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) + { + 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); + } + 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(database_name)) + databases.emplace(database_name, std::filesystem::path(path) / escapeForFileName(database_name)); + } + } + TablesLoader::Databases loaded_databases; for (const auto & [name, db_path] : databases) { 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 +#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; + extern const int TABLE_ALREADY_EXISTS; +} + +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, + StageSeq & stage_seq, + std::stack & leaf_nodes) +{ + /// 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) + { + result.is_shuffle = true; + LOG_DEBUG(log, "[{}]Check shuffle: {}, current node is UnionStep", getStageSeqName(stage_seq), result.is_shuffle); + stage_seq = StageSeq::STAGE1; + return; + } + + result.current_join_step = dynamic_cast(current_node->step.get()); + if (result.current_join_step) + { + assert(current_node->children.size() == 2); + bool maybe_need_shuffle = false; + const auto join_kind = result.current_join_step->getJoin()->getTableJoin().kind(); + if (isFull((join_kind))) + { + 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 + 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; + } + + result.child_aggregating_step = dynamic_cast(child_node->step.get()); + if (result.child_aggregating_step) + { + /// From : AggregatingStep => + /// To : AggregatingStep(partial) [=> MergingAggregatedStep(final)] => + 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; + } + + 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) => + 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; + } + + if ((result.child_distinct_step = dynamic_cast(child_node->step.get()))) + { + 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; + } + + if ((result.child_limit_step = dynamic_cast(child_node->step.get()))) + { + 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; + } +} + +void DistributedPlanner::transferInterpreterParams(QueryPlan::Node *current_node, QueryPlan::Node *last_node) +{ + if (last_node->interpreter_params) + { + if (!current_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 {}", + 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 if (current_node->children.size() > 1) + { + /// 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 {}", + current_node->step->getName(), + static_cast(current_node->interpreter_params.get()), + current_node->interpreter_params->group_by_with_totals, + last_node->step->getName(), + 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; + } + } +} + +void DistributedPlanner::buildStages() +{ + LOG_DEBUG(log, "===> Build stages."); + + 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(); + + if (root_node) + { + /// 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[i] = parent_stages.top(); + new_stage->parents[i]->child = new_stage; + parent_stages.pop(); + } + /// 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[i] = leaf_nodes.top(); + /// This leaf node is a data source node reading data from storage. + if (new_stage->leaf_nodes[i]->children.empty()) + new_stage->is_leaf_stage = true; + leaf_nodes.pop(); + } + } + 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; + }; + + 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, stage_seq, leaf_nodes); + if (result.is_shuffle) + { + ++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); + 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. + transferInterpreterParams(frame.node, last_node); + + ++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: {}({})", getStageSeqName(stage_seq), frame.node->step->getName(), frame.node->step->getStepDescription()); + 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. + /// 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{ + .step = std::move(step), + .children = {last_node}, + .num_parent_stages = 1, + .interpreter_params = last_node->interpreter_params}); + query_plan.root = &query_plan.nodes.back(); + LOG_DEBUG( + log, + "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. + 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::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. + 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 special_storages{"HDFS", "S3", "MySQL", "Memory"}; + + 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 (isSinglePointDataSource(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(); + plan_result.is_result_stage_moved_forward = true; + return; + } + } + LOG_DEBUG(log, "Schedule stage {} to 1 worker(local).", stage->id); + return; + } + + /// Intermediate stage. + stage->workers.emplace_back(std::make_shared(my_replica)); + LOG_DEBUG(log, "Schedule stage {} to 1 worker(local).", stage->id); + }; + + 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 (plan_result.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); + + /// 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 + { + /// 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, + 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); + } + } + 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); + 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)): + 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, "initiator"); + 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()); + } + } +} + +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 {}({} 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 (size_t i = 0; !leaf_nodes.empty() && i < root_node->num_leaf_nodes_in_stage; ++i) + { + leaf_nodes.pop(); + } + } + }; + + 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; + 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()) + { + auto & frame = stack.top(); + + if (one_child_is_visited) + { + /// Transfer interpreter params bottom-up. + transferInterpreterParams(frame.node, last_node); + + CheckShuffleResult result; + 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); + 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, &last_node, &stage_seq](QueryPlanStepPtr step, const String & description, QueryPlan::Node * & node) + { + 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 = last_node->interpreter_params}); + else + { + 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; + 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 + : ((result.child_aggregating_step->getParams().optimize_trivial_count && result.grandchild_step) + ? result.grandchild_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, + last_node->interpreter_params->group_by_with_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 + + /// If current step is JoinStep or UnionStep, only add DistributedSourceStep. + + /// If parent stage has aggregate, add MergingAggregatedStep. + if (result.child_aggregating_step) + { + 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); + 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. + 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. + else if (result.child_limit_step) + { + if (result.grandchild_sorting_step) + { + auto merging_sorted + = 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); + } + + 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, &stage_seq](QueryPlanStepPtr step, QueryPlan::Node * & node) + { + LOG_DEBUG(log, "[{}]Replace step: {}, stage: {}", getStageSeqName(stage_seq), 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); + if (query_distributed_plan_info.empty_result_for_aggregation_by_empty_set) + { + 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); + } + replaceStep(std::move(aggregating_step), last_node); + } + /// If optimize trivial count, remove AggregatingStep. + else + { + LOG_DEBUG(log, "[{}]Remove step: {}, stage: {}", getStageSeqName(stage_seq), 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; + } + + /// 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(), frame.node->step->getStepDescription()); + 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(); + + 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())) + { + /// This will change query_plan.root, so save new root. + uniteCreatingSetSteps(creating_set_plans); + result_stage->root_node = query_plan.root; + } + + 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; + 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, "Local 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..cce94751a39b --- /dev/null +++ b/src/Processors/QueryPlan/DistributedPlanner.h @@ -0,0 +1,121 @@ +#pragma once + +#include +#include + +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); + + 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. + 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; + 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. + 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; + 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, + StageSeq & stage_seq, + std::stack & leaf_nodes); + + struct PlanResult + { + String initial_query_id; + 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 transferInterpreterParams(QueryPlan::Node *current_node, QueryPlan::Node *last_node); + void buildStages(); + void debugStages(); + + void 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/DistributedSourceStep.cpp b/src/Processors/QueryPlan/DistributedSourceStep.cpp new file mode 100644 index 000000000000..3e7308aad60b --- /dev/null +++ b/src/Processors/QueryPlan/DistributedSourceStep.cpp @@ -0,0 +1,83 @@ +#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_, + const String & query_id_, + int stage_id_, + 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) + , sources(sources_) + , query_id(query_id_) + , stage_id(stage_id_) + , 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, 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); + addConvertingActions(pipes.back(), output_stream->header); +} + +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..fc9c69587933 --- /dev/null +++ b/src/Processors/QueryPlan/DistributedSourceStep.h @@ -0,0 +1,57 @@ +#pragma once +#include +#include +#include + +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, + false, + context_) + { + } + + DistributedSourceStep( + Block header_, + const std::vector> & sources_, + const String & query_id_, + int stage_id_, + int parent_stage_id_, + const String & node_id_, + bool add_aggregation_info_, + bool add_totals_, + ContextPtr context_); + + 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; } + + 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; + bool add_aggregation_info; + bool add_totals; + ContextPtr context; + Poco::Logger * log; +}; +} + diff --git a/src/Processors/QueryPlan/LimitStep.h b/src/Processors/QueryPlan/LimitStep.h index f5bceeb29c7a..c939c0d28282 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; @@ -36,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/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/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index f319e562bfbd..45156b4c3e7c 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -10,17 +10,35 @@ #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; +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; @@ -49,7 +67,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, InterpreterParamsPtr interpreter_params) { if (isInitialized()) throw Exception("Cannot unite plans because current QueryPlan is already initialized", @@ -62,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) @@ -71,19 +89,22 @@ 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); } for (auto & plan : plans) nodes.splice(nodes.end(), std::move(plan->nodes)); - nodes.emplace_back(Node{.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) + { root->children.emplace_back(plan->root); + plan->root->parent = root; + } for (auto & plan : plans) { @@ -93,7 +114,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)}); + "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; } @@ -114,18 +135,19 @@ void QueryPlan::addStep(QueryPlanStepPtr step) { 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}}); - root = &nodes.back(); + nodes.emplace_back(Node{.step = std::move(step), .children = {root}, .interpreter_params = std::move(interpreter_params)}); + root->parent = &nodes.back(); + root = root->parent; return; } @@ -134,6 +156,29 @@ void QueryPlan::addStep(QueryPlanStepPtr step) " input expected", ErrorCodes::LOGICAL_ERROR); } +void QueryPlan::reset() +{ + root = nullptr; + nodes.clear(); +} + +void QueryPlan::collectCreatingSetPlan(std::vector> & creating_set_plans) +{ + if (creating_set_plans.empty() && dynamic_cast(root->step.get())) + { + LOG_DEBUG(log, "Collect {} CreatingSetStep", root->children.size() - 1); + for (auto * child : root->children) + { + if (dynamic_cast(child->step.get())) + { + auto plan = std::make_unique(); + plan->root = child; + creating_set_plans.emplace_back(std::move(plan)); + } + } + } +} + 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..4f190f99627e 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -36,6 +37,31 @@ 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; + } + + 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; + 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. @@ -47,15 +73,21 @@ 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, 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() const DataStream & getCurrentDataStream() const; /// Checks that (isInitialized() && !isCompleted()) + void checkInitialized() const; + void checkNotCompleted() const; void optimize(const QueryPlanOptimizationSettings & optimization_settings); + void reset(); + + void collectCreatingSetPlan(std::vector> & creating_set_plans); + QueryPipelineBuilderPtr buildQueryPipeline( const QueryPlanOptimizationSettings & optimization_settings, const BuildQueryPipelineSettings & build_pipeline_settings); @@ -100,20 +132,25 @@ class QueryPlan { QueryPlanStepPtr step; std::vector children = {}; + Node * parent = nullptr; + 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; }; using Nodes = std::list; private: + friend class DistributedPlanner; + 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; + + Poco::Logger * log; }; std::string debugExplainStep(const IQueryPlanStep & step); 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/Processors/Sources/DistributedSource.cpp b/src/Processors/Sources/DistributedSource.cpp new file mode 100644 index 000000000000..af91f78a72a5 --- /dev/null +++ b/src/Processors/Sources/DistributedSource.cpp @@ -0,0 +1,150 @@ +#include +#include +#include +#include +#include + +namespace DB +{ +DistributedSource::DistributedSource(DistributedSourceExecutorPtr executor, bool add_aggregation_info_, bool async_read_) + : SourceWithProgress(executor->getHeader()) + , query_executor(std::move(executor)) + , add_aggregation_info(add_aggregation_info_) + , async_read(async_read_) + , log(&Poco::Logger::get("DistributedSource")) +{ + /// 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; + +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); + + 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); +} + +void DistributedSource::onCancel() +{ + was_query_canceled = true; + query_executor->cancel(); +} + +void DistributedSource::onUpdatePorts() +{ + if (getPort().isFinished()) + { + was_query_canceled = true; + query_executor->finish(); + } +} + +DistributedTotalsSource::DistributedTotalsSource(DistributedSourceExecutorPtr executor_) + : ISource(executor_->getHeader()) + , executor(std::move(executor_)) +{ +} + +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 new file mode 100644 index 000000000000..d267bccd07aa --- /dev/null +++ b/src/Processors/Sources/DistributedSource.h @@ -0,0 +1,72 @@ +#pragma once +#include +#include + +namespace DB +{ +class DistributedSourceExecutor; +using DistributedSourceExecutorPtr = std::shared_ptr; + +class DistributedSource : public SourceWithProgress +{ +public: + DistributedSource(DistributedSourceExecutorPtr executor, + bool add_aggregation_info_, + bool async_read_); + ~DistributedSource() override; + + Status prepare() override; + String getName() const override { return "DistributedSource"; } + + void onUpdatePorts() override; + +protected: + std::optional tryGenerate() override; + void onCancel() override; + +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; +}; + +/// 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. +/// 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/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/QueryPipeline/DistributedSourceExecutor.cpp b/src/QueryPipeline/DistributedSourceExecutor.cpp new file mode 100644 index 000000000000..2ae9a69224f5 --- /dev/null +++ b/src/QueryPipeline/DistributedSourceExecutor.cpp @@ -0,0 +1,122 @@ +#include +#include + +namespace DB +{ +DistributedSourceExecutor::DistributedSourceExecutor( + const Block & header_, + const std::shared_ptr & source_, + const String & query_id_, + const String & node_id_, + 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_) + , 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(parent_stage_id); + ticket.set_node_id(node_id); + client.prepareRead(ticket); +} + +DistributedSourceExecutor::~DistributedSourceExecutor() +{ +} + +Block DistributedSourceExecutor::read() +{ + if (was_cancelled) + return Block(); + + try + { + Block block; + auto message_type = client.read(block); + switch (message_type) + { + case GRPCClient::MessageType::Data: + if (!block) + finished = true; + 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; + +// default: +// throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown GRPC block type"); + } + return {}; + } + catch (...) + { + got_exception_from_replica = true; + throw; + } +} + +void DistributedSourceExecutor::finish() +{ + if (!isQueryPending() || hasThrownException()) + return; + + LOG_DEBUG(log, "Finish reading."); + tryCancel("Enough data has been read"); + + finished = true; +} + +void DistributedSourceExecutor::cancel() +{ + if (!isQueryPending() || hasThrownException()) + return; + + LOG_DEBUG(log, "Cancel reading."); + client.cancel(); + tryCancel("Query is cancelled"); +} + +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, "Reason: {}.", 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..df441ee2b651 --- /dev/null +++ b/src/QueryPipeline/DistributedSourceExecutor.h @@ -0,0 +1,58 @@ +#pragma once +#include +#include +#include + +namespace DB +{ +class DistributedSourceExecutor +{ +public: + /// Takes already set connection. + DistributedSourceExecutor( + const Block & header_, + const std::shared_ptr & source_, + const String & query_id_, + const String & node_id_, + int stage_id_, + int parent_stage_id_); + + ~DistributedSourceExecutor(); + + Block read(); + void finish(); + void cancel(); + + 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; + int stage_id; + int parent_stage_id; + 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..7ca661d651c6 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 @@ -43,9 +44,12 @@ #include #include +#include +using namespace std::chrono_literals; 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; @@ -214,15 +218,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("\""); @@ -272,6 +278,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 +336,9 @@ namespace CALL_WITH_STREAM_INPUT, /// ExecuteQueryWithStreamInput() call 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_CANCEL_PLAN_FRAGMENT, /// CancelPlanFragment() call CALL_MAX, }; @@ -340,6 +350,9 @@ namespace case CALL_WITH_STREAM_INPUT: return "ExecuteQueryWithStreamInput()"; case CALL_WITH_STREAM_OUTPUT: return "ExecuteQueryWithStreamOutput()"; 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(); @@ -352,7 +365,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_FETCH_PLAN_FRAGMENT_RESULT); } template @@ -379,6 +392,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 +429,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 +469,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 +506,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 +525,129 @@ 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.RequestExecutePlanFragment(&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.RequestFetchPlanFragmentResult(&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; + }; + + 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) @@ -500,6 +656,9 @@ 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_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(); @@ -562,12 +721,214 @@ namespace mutable std::condition_variable changed; }; + template + class InnerMap + { + public: + InnerMap(String name): log(&Poco::Logger::get(name)) {} + + 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 std::pair{{}, false}; + return std::pair{it->second, true}; + } + + auto insert(const String & key_, Value && value_) + { + std::lock_guard lock(mutex); + return container.emplace(key_, value_); + } + + auto erase(const String & key_) + { + std::lock_guard lock(mutex); + 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 + { + public: + enum Status { + NORMAL, + TIMEOUT, + FINISH, + CANCEL + }; + + QueryInfoWrapper(const std::shared_ptr & 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(); + + std::shared_ptr 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 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); @@ -576,22 +937,33 @@ namespace void run(); void receiveQuery(); + void receiveTicket(); void executeQuery(); + void storeQueryInfoWrapper(); + void loadQueryInfoWrapper(bool is_cancel = false); + void processInput(); void initializeBlockInputStream(const Block & header); void createExternalTables(); void generateOutput(); + void produceOutput(); + void consumeOutput(); + void cancelPlanFragment(); void finishQuery(); + void finishQueryInfo(); + void finishPipeline(); void onException(const Exception & exception); void onFatalError(); void releaseQueryIDAndSessionID(); void close(); void readQueryInfo(); + void readTicket(); void throwIfFailedToReadQueryInfo(); + void throwIfFailedToReadTicket(); bool isQueryCancelled(); void addProgressToResult(); @@ -606,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; @@ -625,10 +1001,12 @@ 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; bool initial_query_info_read = false; + bool initial_ticket_read = false; bool finalize = false; bool responder_finished = false; bool cancelled = false; @@ -648,8 +1026,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; - GRPCQueryInfo next_query_info_while_reading; + std::atomic failed_to_read_ticket = false; + 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; BoolState sending_result{false}; @@ -658,8 +1039,19 @@ namespace ThreadFromGlobalPool call_thread; }; - 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_) { } @@ -690,12 +1082,48 @@ namespace { try { - setThreadName("GRPCServerCall"); - receiveQuery(); - executeQuery(); - processInput(); - generateOutput(); - finishQuery(); + if (call_type == CALL_EXECUTE_PLAN_FRAGMENT) + { + setThreadName("GRPCExecPlan"); + + /// 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("GRPCFetchResult"); + receiveTicket(); + loadQueryInfoWrapper(); + executeQuery(); + consumeOutput(); + finishQuery(); + } + else if (call_type == CALL_CANCEL_PLAN_FRAGMENT) + { + setThreadName("GRPCCancelPlan"); + receiveTicket(); + loadQueryInfoWrapper(true); + if (query_info_wrapper) + cancelPlanFragment(); + finishQuery(); + } + else + { + setThreadName("GRPCServerCall"); + receiveQuery(); + executeQuery(); + processInput(); + generateOutput(); + finishQuery(); + } } catch (Exception & exception) { @@ -717,18 +1145,32 @@ 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() + { + LOG_INFO(log, "Handling call {}", getCallName(call_type)); + + readTicket(); + + LOG_DEBUG(log, "Received ticket: {}", ticket.initial_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(); - 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()) @@ -744,24 +1186,52 @@ 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(); + /// Set query plan fragment info + if (call_type == CALL_EXECUTE_PLAN_FRAGMENT) + { + std::unordered_map>> parent_sources; + for (const auto & parent : query_info->parent_sources()) + { + 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)); + } + 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, + .empty_result_for_aggregation_by_empty_set = query_info->empty_result_for_aggregation_by_empty_set()}; + query_context->setQueryPlanFragmentInfo(std::move(fragment_info)); + } + /// 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->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. @@ -777,19 +1247,26 @@ 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; 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())); + query_text = std::move(*(query_info->mutable_query())); const char * begin = query_text.data(); const char * end = begin + query_text.size(); ParserQuery parser(end); @@ -804,10 +1281,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) { @@ -824,25 +1301,53 @@ 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_context = query_context->getInitialContext(plan_fragment_id); + if (query_info->has_view_source()) + { + 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_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; @@ -854,13 +1359,51 @@ namespace io = ::DB::executeQuery(true, query, query_context); } + 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())); + if (!res.second) + { + 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) + { + 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 + { + 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 + throw Exception("Query info key " + query_info_key + " not exists", ErrorCodes::LOGICAL_ERROR); + } + + } + void Call::processInput() { if (!io.pipeline.pushing()) 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) @@ -905,30 +1448,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", @@ -938,7 +1481,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; } @@ -980,7 +1523,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()) @@ -1010,6 +1553,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()) @@ -1050,24 +1594,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", @@ -1075,7 +1619,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()); } } @@ -1172,12 +1716,156 @@ 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)) + 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(); + LOG_DEBUG(log, "{} produce totals {}, extremes {}.", query_info_key, query_info_wrapper->totals.rows(), query_info_wrapper->extremes.rows()); + } + query_info_wrapper->notifyFinish(); + } + } + + 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(); + } + + /// Throw exception which is from producer. + 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) + { + 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); + } + + 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::cancelPlanFragment() + { + /// Cancel plan fragment, including producer and consumer. + query_info_wrapper->cancel = true; + result.set_cancelled(true); + } + void Call::finishQuery() { finalize = true; io.onFinish(); addProgressToResult(); - query_scope->logPeakMemoryUsage(); + if (query_scope.has_value()) + { + query_scope->logPeakMemoryUsage(); + } addLogsToResult(); releaseQueryIDAndSessionID(); sendResult(); @@ -1192,8 +1880,51 @@ 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) { + if (call_type == CALL_EXECUTE_PLAN_FRAGMENT) + { + if (query_info_wrapper) + { + LOG_DEBUG(log, "{} producer has an exception", query_info_key); + 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(); + } + } + io.onException(); LOG_ERROR(log, getExceptionMessage(exception, true)); @@ -1264,6 +1995,16 @@ namespace query_scope.reset(); query_context.reset(); session.reset(); + 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() @@ -1271,12 +2012,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() @@ -1308,7 +2050,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; }; @@ -1329,6 +2071,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 +2117,14 @@ namespace } } + void Call::throwIfFailedToReadTicket() + { + if (failed_to_read_ticket) + { + throw Exception("Failed to read Ticket", ErrorCodes::NETWORK_ERROR); + } + } + bool Call::isQueryCancelled() { if (cancelled) @@ -1536,12 +2321,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() @@ -1561,6 +2353,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(); } @@ -1607,7 +2412,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); }); @@ -1658,6 +2463,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; @@ -1665,6 +2493,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; }; diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index c6cafaf6e404..eac3ad7567a3 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -60,6 +60,18 @@ message Compression { CompressionLevel level = 2; } +message Ticket +{ + // same as QueryInfo.query_id + string initial_query_id = 1; + int32 stage_id = 2; + 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. @@ -101,6 +113,19 @@ 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; + + uint32 query_kind = 18; + /// Below are used in distributed plan execution. + string initial_query_id = 19; + int32 stage_id = 20; + string node_id = 21; + /// The mapping of parent stage id to sources. + map parent_sources = 22; + /// Format is "replica_name:grpc_port" + 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 { @@ -171,4 +196,8 @@ service ClickHouse { rpc ExecuteQueryWithStreamInput(stream QueryInfo) returns (Result) {} rpc ExecuteQueryWithStreamOutput(QueryInfo) returns (stream Result) {} rpc ExecuteQueryWithStreamIO(stream QueryInfo) returns (stream Result) {} + + rpc ExecutePlanFragment(QueryInfo) returns (Result) {} + rpc FetchPlanFragmentResult(Ticket) returns (stream Result) {} + rpc CancelPlanFragment(Ticket) returns (Result) {} } 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..cc689d9fb27a 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -127,13 +127,13 @@ 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) 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/IStorage.cpp b/src/Storages/IStorage.cpp index dd851f19906d..4733483beb41 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -115,12 +115,14 @@ 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 { auto read_step = std::make_unique(std::move(pipe), getName()); - query_plan.addStep(std::move(read_step)); + 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)); } } 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 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; } 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); } diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 982acfe62a43..163090ea0df7 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, @@ -816,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/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_helpergetLocalShardCount() + 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,73 @@ 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); + + 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, + args.constraints, + args.comment, + args.table_id.database_name, + args.table_id.table_name, + "store", + context, + nullptr, + storage_policy, + 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/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/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())); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7322d57fed2f..e0640a8222bb 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -67,7 +68,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 +85,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 +233,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 && getStorageID().getDatabaseName() != DatabaseCatalog::SYSTEM_DATABASE) + { + 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); @@ -1499,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 8ed4b707b347..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; @@ -137,6 +141,8 @@ class StorageMergeTree final : public shared_ptr_helper, publi std::atomic shutdown_called {false}; + StoragePtr embedded_distributed; + private: void loadMutations(); @@ -236,7 +242,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; }; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f4a50f2e5531..32be237a6e88 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include @@ -274,7 +275,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 +307,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 +4318,11 @@ void StorageReplicatedMergeTree::assertNotReadonly() const SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { + if (getContext()->getRunningMode() == Context::RunningMode::COMPUTE && getStorageID().getDatabaseName() != DatabaseCatalog::SYSTEM_DATABASE) + { + return embedded_distributed->write(nullptr, metadata_snapshot, local_context); + } + const auto storage_settings_ptr = getSettings(); assertNotReadonly(); @@ -4323,7 +4331,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, @@ -5032,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 @@ -5053,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 65daf82a6338..f2134c2bb262 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -146,6 +146,8 @@ class StorageReplicatedMergeTree final : public shared_ptr_helper void foreachCommittedParts(Func && func, bool select_sequential_consistency) const; @@ -743,7 +747,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);